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 2021/05/11 14:19:23 UTC

[GitHub] [accumulo] dlmarion opened a new pull request #2096: External Compactions

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


   closes #1451 by adding two new optional components to support running compactions outside of the TabletServer


-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   > @milleruntime @ctubbsii - I'm looking for an existing IT that tests encryption so I can write something for the external compactions. I have found CryptoTest, which is a unit test, and WriteAheadLogEncryptedIT which doesn't have anything that might be relevant for me. Do you know of an existing test that I can look to for an example?
   
   That is currently all we have for automated tests. The other testing we did was manual using Uno, which has a configurable plugin for crypto. 


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   @ctubbsii wrote:
   > it seems that the way the external compactions are exposed through to the user in the compaction configuration is through the specially named externalQueue
   
   The DefaultCompactionPlanner javadoc is out of date, it's doesn't describe `externalQueue`. Looking at `DefaultCompactionPlanner.ExecutorConfig` and `DefaultCompactionPlanner.init()` it seems it would be trivial to change the configuration so that it was more explicit. We could add a `type` field, for example:
   
   ```
   [
    {
      "name": "small",
      "type": "internal",
      "maxSize": "16M",
      "numThreads": 4
    },
    {
      "name": "large",
      "type": "external",
      "queue": "LargeQ"
    }
   ]
   ```


-- 
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.

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



[GitHub] [accumulo] milleruntime commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632593672



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(
+      Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions, Tablet tablet,
+      Map<ExternalCompactionId,String> externalCompactionsToRemove) {
+    CompactionKind extKind = null;
+    boolean unexpectedExternal = false;
+    Set<StoredTabletFile> tmpSelectedFiles = null;
+    Boolean selAll = null;
+    Long cid = null;
+    Boolean propDel = null;
+    int count = 0;
+
+    ArrayList<String> reasons = new ArrayList<>();
+
+    for (Entry<ExternalCompactionId,ExternalCompactionMetadata> entry : extCompactions.entrySet()) {

Review comment:
       Breaking this method up into smaller methods would make it a lot easier to grok. Perhaps one for the information gathering in this loop.

##########
File path: core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.schema;
+
+import java.util.Base64;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+public class ExternalCompactionFinalState {
+
+  private static final Gson GSON = new GsonBuilder().create();
+
+  public enum FinalState {
+    FINISHED, FAILED
+  }
+
+  private ExternalCompactionId ecid;
+  private KeyExtent extent;
+  private FinalState state;
+  private long fileSize;
+  private long fileEntries;
+
+  public ExternalCompactionFinalState(ExternalCompactionId ecid, KeyExtent extent, FinalState state,
+      long fileSize, long fileEntries) {
+    this.ecid = ecid;
+    this.extent = extent;
+    this.state = state;
+    this.fileSize = fileSize;
+    this.fileEntries = fileEntries;
+  }
+
+  public ExternalCompactionId getExternalCompactionId() {
+    return ecid;
+  }
+
+  public FinalState getFinalState() {
+    return state;
+  }
+
+  public KeyExtent getExtent() {
+    return extent;
+  }
+
+  public long getFileSize() {
+    Preconditions.checkState(state == FinalState.FINISHED);
+    return fileSize;
+  }
+
+  public long getEntries() {
+    Preconditions.checkState(state == FinalState.FINISHED);
+    return fileEntries;
+  }
+
+  private static class Extent {
+
+    String tableId;
+    String er;
+    String per;
+
+    Extent(KeyExtent extent) {
+      this.tableId = extent.tableId().canonical();
+      if (extent.endRow() != null) {
+        er = Base64.getEncoder().encodeToString(TextUtil.getBytes(extent.endRow()));
+      }
+
+      if (extent.prevEndRow() != null) {
+        per = Base64.getEncoder().encodeToString(TextUtil.getBytes(extent.prevEndRow()));
+      }

Review comment:
       I think these could be final:
   ```suggestion
       final String tableId;
       final String er;
       final String per;
   
       Extent(KeyExtent extent) {
         this.tableId = extent.tableId().canonical();
         if (extent.endRow() != null) {
           er = Base64.getEncoder().encodeToString(TextUtil.getBytes(extent.endRow()));
         } else {
           er = null;
         }
   
         if (extent.prevEndRow() != null) {
           per = Base64.getEncoder().encodeToString(TextUtil.getBytes(extent.prevEndRow()));
         } else {
           per = null;
         }
   ```

##########
File path: core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java
##########
@@ -31,4 +31,9 @@
    * Create a thread pool executor within a compaction service.
    */
   public CompactionExecutorId createExecutor(String name, int threads);
+
+  /**
+   * @return an id for a configured external execution queue.
+   */
+  public CompactionExecutorId getExternalExecutor(String name);

Review comment:
       I don't think we need the `ExecutorManager` interface. These methods could exist on the `InitParameters` interface. If a user wrote their own `ExecutorManager` it doesn't look like there is a way for Accumulo to use it since CompactionService is always calling the internal one.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java
##########
@@ -40,14 +42,25 @@
   private final Set<CompactableFile> files;
   private final CompactionKind kind;
   private boolean selectedAll;
+  private boolean hasSelectedAll;
 
-  CompactionJobImpl(long priority, CompactionExecutorId executor, Collection<CompactableFile> files,
-      CompactionKind kind, boolean selectedAllFiles) {
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind, boolean selectedAllFiles) {
     this.priority = priority;
     this.executor = Objects.requireNonNull(executor);
     this.files = Set.copyOf(files);
-    this.kind = kind;
+    this.kind = Objects.requireNonNull(kind);
     this.selectedAll = selectedAllFiles;
+    this.hasSelectedAll = true;
+  }
+
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind) {
+    this.priority = priority;
+    this.executor = Objects.requireNonNull(executor);
+    this.files = Set.copyOf(files);
+    this.kind = Objects.requireNonNull(kind);
+    this.hasSelectedAll = false;

Review comment:
       Looks like the new constructor is just to set `hasSelectedAll` to false, throwing the error. There is another `selectedAll` in CompactableImpl that is set as well but that one is Boolean object so could be null? It looks like this `selectedAll` is purely to set `propogateDeletes` so maybe it could be renamed to that? But in `ExternalCompactionMetadata` there are two booleans, `selectAll` and `propogateDeletes`? Needless to say I'm confused and lost.

##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -539,54 +766,58 @@ public boolean isCompactionEnabled() {
     }
   }
 
-  @Override
-  public void compact(CompactionServiceId service, CompactionJob job, RateLimiter readLimiter,
-      RateLimiter writeLimiter, long queuedTime) {
-
-    Set<StoredTabletFile> jobFiles = job.getFiles().stream()
-        .map(cf -> ((CompactableFileImpl) cf).getStoredTabletFile()).collect(Collectors.toSet());
-
-    Long compactionId = null;
+  private static class CompactionInfo {
+    Set<StoredTabletFile> jobFiles;
     Long checkCompactionId = null;
     boolean propogateDeletes = true;
     CompactionHelper localHelper;
     List<IteratorSetting> iters = List.of();
     CompactionConfig localCompactionCfg;
+    boolean selectedAll;
+    Set<StoredTabletFile> selectedFiles;
+  }
+
+  private CompactionInfo reserveFilesForCompaction(CompactionServiceId service, CompactionJob job) {

Review comment:
       Would be helpful to mention this method can return null and why. I am not sure using `Optional` would help but it is a thought.

##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -95,6 +117,8 @@
 
   private Supplier<Set<CompactionServiceId>> servicesInUse;
 
+  private Set<CompactionServiceId> servicesUsed = new ConcurrentSkipListSet<>();
+
   // status of special compactions
   private enum SpecialStatus {
     NEW, SELECTING, SELECTED, NOT_ACTIVE, CANCELED

Review comment:
       This enum name is a bit obscure. It looks like its only used for the status of file selection. Maybe `FileSelectionStatus` might make more sense

##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(

Review comment:
       There is a lot of checking going on in this method. At least one comment describing some high level goals would be very helpful.

##########
File path: core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionFinalState.java
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.schema;
+
+import java.util.Base64;
+
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.util.TextUtil;
+import org.apache.hadoop.io.Text;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+
+public class ExternalCompactionFinalState {
+
+  private static final Gson GSON = new GsonBuilder().create();
+
+  public enum FinalState {
+    FINISHED, FAILED
+  }
+
+  private ExternalCompactionId ecid;
+  private KeyExtent extent;
+  private FinalState state;
+  private long fileSize;
+  private long fileEntries;

Review comment:
       These fields could be final
   ```suggestion
     private final ExternalCompactionId ecid;
     private final KeyExtent extent;
     private final FinalState state;
     private final long fileSize;
     private final long fileEntries;
   ```




-- 
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.

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



[GitHub] [accumulo] milleruntime commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632067839



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.compaction;
+
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {

Review comment:
       I get that. Thinking ahead is good but why make it more awkward right now if we don't have to? I was also thinking in this case having those "i" and "e" prefixes in the SPI would be good and there isn't much to `CompactionExecutorId`




-- 
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.

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



[GitHub] [accumulo] DomGarguilo commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635323610



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
##########
@@ -534,10 +540,10 @@ public static AccumuloConfiguration getCompactionConfig(CompactionKind kind, Tab
   }
 
   static StoredTabletFile compact(Tablet tablet, CompactionJob job, Set<StoredTabletFile> jobFiles,
-      Long compactionId, boolean propogateDeletes, CompactableImpl.CompactionHelper helper,
-      List<IteratorSetting> iters, CompactionCheck compactionCheck, RateLimiter readLimiter,
-      RateLimiter writeLimiter, CompactionStats stats)
-      throws IOException, CompactionCanceledException {
+      Long compactionId, Set<StoredTabletFile> selectedFiles, boolean propogateDeletes,

Review comment:
       ```suggestion
         Long compactionId, Set<StoredTabletFile> selectedFiles, boolean propagateDeletes,
   ```




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > These changes help, thanks. I am still wondering why we need both constructors? If you make the jobSelectedAll an Optional type parameter wouldn't that suffice?
   
   @milleruntime  I made the constructor take an optional and added a comment about the parameter in 28c84b6


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633923100



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.compactor;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator;
+import org.apache.accumulo.core.compaction.thrift.Compactor.Iface;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
+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.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.trace.TraceUtil;
+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.ServerServices;
+import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.core.util.threads.Threads;
+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.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.ServerOpts;
+import org.apache.accumulo.server.compaction.CompactionInfo;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.hadoop.fs.Path;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class Compactor extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.Compactor.Iface {
+
+  public static class CompactorServerOpts extends ServerOpts {
+    @Parameter(required = true, names = {"-q", "--queue"}, description = "compaction queue name")
+    private String queueName = null;
+
+    public String getQueueName() {
+      return queueName;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(Compactor.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long TIME_BETWEEN_CANCEL_CHECKS = 5 * 60 * 1000;
+
+  private static final long TEN_MEGABYTES = 10485760;
+  private static final CompactionCoordinator.Client.Factory COORDINATOR_CLIENT_FACTORY =
+      new CompactionCoordinator.Client.Factory();
+
+  protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  private final UUID compactorId = UUID.randomUUID();
+  private final AccumuloConfiguration aconf;
+  private final String queueName;
+  private final AtomicReference<CompactionCoordinator.Client> coordinatorClient =
+      new AtomicReference<>();
+  protected final AtomicReference<ExternalCompactionId> currentCompactionId =
+      new AtomicReference<>();
+
+  private SecurityOperation security;
+  private ServiceLock compactorLock;
+  private ServerAddress compactorAddress = null;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  protected Compactor(CompactorServerOpts opts, String[] args) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = getConfiguration();
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected Compactor(CompactorServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = conf;
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor,
+      long timeBetweenChecks) {
+    schedExecutor.scheduleWithFixedDelay(() -> checkIfCanceled(), 0, timeBetweenChecks,
+        TimeUnit.MILLISECONDS);
+  }
+
+  protected void checkIfCanceled() {
+    TExternalCompactionJob job = JOB_HOLDER.getJob();
+    if (job != null) {
+      try {
+        var extent = KeyExtent.fromThrift(job.getExtent());
+        var ecid = ExternalCompactionId.of(job.getExternalCompactionId());
+
+        TabletMetadata tabletMeta =
+            getContext().getAmple().readTablet(extent, ColumnType.ECOMP, ColumnType.PREV_ROW);
+        if (tabletMeta == null || !tabletMeta.getExtent().equals(extent)
+            || !tabletMeta.getExternalCompactions().containsKey(ecid)) {
+          // table was deleted OR tablet was split or merged OR tablet no longer thinks compaction
+          // is running for some reason
+          LOG.info("Cancelling compaction {} that no longer has a metadata entry at {}", ecid,

Review comment:
       @ctubbsii I Am currently working through a complete review of the code.  You had mentioned having a very generic per tserver plugin to generally handle external compactions.  Thinking about that when looking at this code, it cancels a running external compaction for a tablet that was deleted (split/merge/table deletion).  To handle this case in a highly pluggable environment, this functionality would need to be present in some form in the SPI.   This would be unrelated to any pluggable component on a tserver, its more related to running external compactions.  The tablet for a running external compaction could be unload and then the tablet deleted, no tserver would be aware of 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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633708285



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }
+
+          // As a fallback, try them all
+          if (!matchFound) {
+            LOG.debug("Checking all tservers for external running compaction, extent: {}", extent);
+            for (TServerInstance tsi : tservers) {
+              TabletClientService.Client client = null;
+              try {
+                client = getTabletServerConnection(tsi);
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                }
+              } catch (TException e) {
+                LOG.error(
+                    "Error from tserver {} while trying to check if external compaction is running, trying next tserver",
+                    ExternalCompactionUtil.getHostPortString(tsi.getHostAndPort()), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            }
+          }
+
+          if (!matchFound) {
+            LOG.warn(
+                "There is an external compaction running on a compactor, but could not find corresponding tablet server. Extent: {}, Compactor: {}, Compaction: {}",
+                extent, hp, job);
+          }
+        });
+      }
+      tservers.clear();
+    } else {
+      LOG.info("No running tablet servers found, continuing startup");
+    }
+
+    tserverSet.startListeningForTabletServerChanges();
+    startDeadCompactionDetector();
+
+    LOG.info("Starting loop to check tservers for compaction summaries");
+    while (!shutdown) {
+      long start = System.currentTimeMillis();
+      tserverSet.getCurrentServers().forEach(tsi -> {
+        try {
+          TabletClientService.Client client = null;
+          try {
+            LOG.debug("Contacting tablet server {} to get external compaction summaries",
+                tsi.getHostPort());
+            client = getTabletServerConnection(tsi);
+            List<TCompactionQueueSummary> summaries =
+                client.getCompactionQueueInfo(TraceUtil.traceInfo(), getContext().rpcCreds());
+            summaries.forEach(summary -> {
+              QueueAndPriority qp =
+                  QueueAndPriority.get(summary.getQueue().intern(), summary.getPriority());
+              synchronized (qp) {
+                TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(qp.getQueue(), k -> 0L);
+                QUEUE_SUMMARIES.update(tsi, summaries);
+              }
+            });
+          } finally {
+            ThriftUtil.returnClient(client);
+          }
+        } catch (TException e) {
+          LOG.warn("Error getting external compaction summaries from tablet server: {}",
+              tsi.getHostAndPort(), e);
+          QUEUE_SUMMARIES.remove(Set.of(tsi));
+        }
+      });
+
+      long now = System.currentTimeMillis();
+      TIME_COMPACTOR_LAST_CHECKED.forEach((k, v) -> {
+        if ((now - v) > getMissingCompactorWarningTime()) {
+          LOG.warn("No compactors have checked in with coordinator for queue {} in {}ms", k,
+              getMissingCompactorWarningTime());
+        }
+      });
+
+      long checkInterval = getTServerCheckInterval();
+      long duration = (System.currentTimeMillis() - start);
+      if (checkInterval - duration > 0) {
+        LOG.debug("Waiting {}ms for next tserver check", (checkInterval - duration));
+        UtilWaitThread.sleep(checkInterval - duration);
+      }
+    }
+
+    LOG.info("Shutting down");
+  }
+
+  protected void startDeadCompactionDetector() {
+    new DeadCompactionDetector(getContext(), compactionFinalizer, schedExecutor).start();
+  }
+
+  protected long getMissingCompactorWarningTime() {
+    return FIFTEEN_MINUTES;
+  }
+
+  protected long getTServerCheckInterval() {
+    return this.aconf.getTimeInMillis(Property.COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL);
+  }
+
+  protected TabletMetadata getMetadataEntryForExtent(KeyExtent extent) {
+    return getContext().getAmple().readTablets().forTablet(extent)
+        .fetch(ColumnType.LOCATION, ColumnType.PREV_ROW).build().stream().findFirst().orElse(null);
+  }
+
+  /**
+   * Callback for the LiveTServerSet object to update current set of tablet servers, including ones
+   * that were deleted and added
+   *
+   * @param current
+   *          current set of live tservers
+   * @param deleted
+   *          set of tservers that were removed from current since last update
+   * @param added
+   *          set of tservers that were added to current since last update
+   */
+  @Override
+  public void update(LiveTServerSet current, Set<TServerInstance> deleted,
+      Set<TServerInstance> added) {
+
+    // run() will iterate over the current and added tservers and add them to the internal
+    // data structures. For tservers that are deleted, we need to remove them from QUEUES
+    // and INDEX
+    QUEUE_SUMMARIES.remove(deleted);
+  }
+
+  /**
+   * Return the next compaction job from the queue to a Compactor
+   *
+   * @param queueName
+   *          queue
+   * @param compactorAddress
+   *          compactor address
+   * @return compaction job
+   */
+  @Override
+  public TExternalCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials,
+      String queueName, String compactorAddress, String externalCompactionId) throws TException {
+
+    // do not expect users to call this directly, expect compactors to call this method
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+    final String queue = queueName.intern();
+    LOG.debug("getCompactionJob called for queue {} by compactor {}", queue, compactorAddress);
+    TIME_COMPACTOR_LAST_CHECKED.put(queue, System.currentTimeMillis());
+
+    TExternalCompactionJob result = null;
+
+    PrioTserver prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+
+    while (prioTserver != null) {
+      TServerInstance tserver = prioTserver.tserver;
+
+      LOG.debug("Getting compaction for queue {} from tserver {}", queue, tserver.getHostAndPort());
+      // Get a compaction from the tserver
+      TabletClientService.Client client = null;
+      try {
+        client = getTabletServerConnection(tserver);
+        TExternalCompactionJob job =
+            client.reserveCompactionJob(TraceUtil.traceInfo(), getContext().rpcCreds(), queue,
+                prioTserver.prio, compactorAddress, externalCompactionId);
+        if (null == job.getExternalCompactionId()) {
+          LOG.debug("No compactions found for queue {} on tserver {}, trying next tserver", queue,
+              tserver.getHostAndPort(), compactorAddress);
+
+          QUEUE_SUMMARIES.removeSummary(tserver, queueName, prioTserver.prio);
+          prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+          continue;
+        }
+        RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+            new RunningCompaction(job, compactorAddress, tserver));
+        LOG.debug("Returning external job {} to {}", job.externalCompactionId, compactorAddress);
+        result = job;
+        break;
+      } catch (TException e) {
+        LOG.warn("Error from tserver {} while trying to reserve compaction, trying next tserver",
+            ExternalCompactionUtil.getHostPortString(tserver.getHostAndPort()), e);
+        QUEUE_SUMMARIES.removeSummary(tserver, queueName, prioTserver.prio);
+        prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+      } finally {
+        ThriftUtil.returnClient(client);
+      }
+    }
+
+    if (result == null) {
+      LOG.debug("No tservers found for queue {}, returning empty job to compactor {}", queue,
+          compactorAddress);
+      result = new TExternalCompactionJob();
+    }
+
+    return result;
+
+  }
+
+  /**
+   * Return the Thrift client for the TServer
+   *
+   * @param tserver
+   *          tserver instance
+   * @return thrift client
+   * @throws TTransportException
+   *           thrift error
+   */
+  protected TabletClientService.Client getTabletServerConnection(TServerInstance tserver)
+      throws TTransportException {
+    TServerConnection connection = tserverSet.getConnection(tserver);
+    TTransport transport =
+        ThriftTransportPool.getInstance().getTransport(connection.getAddress(), 0, getContext());
+    return ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+  }
+
+  /**
+   * Return the Thrift client for the Compactor
+   *
+   * @param compactorAddress
+   *          compactor address
+   * @return thrift client
+   * @throws TTransportException
+   *           thrift error
+   */
+  protected Compactor.Client getCompactorConnection(HostAndPort compactorAddress)
+      throws TTransportException {
+    TTransport transport =
+        ThriftTransportPool.getInstance().getTransport(compactorAddress, 0, getContext());
+    return ThriftUtil.createClient(new Compactor.Client.Factory(), transport);
+  }
+
+  /**
+   * Called by the TabletServer to cancel the running compaction.
+   *
+   * @param tinfo
+   *          trace info
+   * @param credentials
+   *          tcredentials object
+   * @param externalCompactionId
+   *          compaction id
+   * @throws TException
+   *           thrift error
+   */
+  @Override
+  public void cancelCompaction(TInfo tinfo, TCredentials credentials, String externalCompactionId)

Review comment:
       Resolved in cdfec2afccd22ea344ad8fd3a3ce622ee5b87324




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > I'm wondering if there are ways this could be made a bit more modular and the configuration a bit more explicit, so that the default compaction service isn't so monolithic, and users can very explicitly swap out the external executor implementation for another one. 
   
   In the current design tablets are authorities, within a distributed system, on what external compactions are running and queued.  There are RPCs to reserve, fail, and commit external compactions within the tserver AND there is an RPC to get summary information about external queues in the tserver.  These RPCs are structured specifically for the way the rest of the distributed system (coordinator and compactors) works.  Should these tserver RPCs be generalized?  If so, how?


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   >  it seems that the way the external compactions are exposed through to the user in the compaction configuration is through the specially named externalQueue, which seems to be a special feature of the default compaction service. Is that correct?
   
   Its a feature of the default compaction planner, not the default compaction service.  Each compaction service has a configured planner that decides what to compact where.  The [default compaction planner](https://github.com/apache/accumulo/blob/3983cd23629789c1ed0020d01fdd968eabfe590f/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java#L73) accepts the config externalQueue.  It then takes this information and calls the SPI method [getExternalExecutor](https://github.com/apache/accumulo/blob/3983cd23629789c1ed0020d01fdd968eabfe590f/core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java#L38).  Anyone could write a compaction planner that uses external queues by calling this same API method.  The planners configuration could work any way the author likes.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > Though, I am not quite sure how to change the log levels for the compactor and coordinator log files. 
   
   @Manno15 you should be able to edit `<ACCUMULO DIR>/conf/log4j2-service.properties` and add something like the following.
   
   ```
   logger.compactor.name = org.apache.accumulo.compactor
   logger.compactor.level = trace
   ```
   
   The above is from a file I had laying around for a bit so not completely sure its still correct.  Could add similar lines for the coordinator. Changes should be picked up while processes are running (which is really really nice).


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633441752



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       Working on 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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633720108



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       To be more specific, my IDE indicated nothing calls [RunningCompaction.getTserver()](https://github.com/apache/accumulo/blob/cdfec2afccd22ea344ad8fd3a3ce622ee5b87324/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java#L58)




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > I'm curious, do the external compactors support RFile encryption and other RFile-specific configs (like block sizes, locality groups, etc.) today? My guess is that they do, because it's still all the same Accumulo code, just running in a different process, but just curious how much you've exercised those while testing this.
   
   The intent is that they should, but not everything has been tested.  The [code that actually runs compactions of files](https://github.com/apache/accumulo/blob/90e76cd59e5ed6e08a75020f0d50fe69c4bebee3/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java#L73) was refactored and moved so that it could be used by tservers and external compactors.  So the same code is used, the question is do we always give it the same inputs.  Also if compactions were to be highly generalized, some analogue of this code would probably need to be exposed via SPI or API.


-- 
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.

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



[GitHub] [accumulo] Manno15 edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844945493


   > @Manno15 I think I may have fixed the issue you were seeing in 3095d3d, if line 1055 was the !closed check. That was an incorrect sanity check, looking at how the close() method works, closed could be true there and that would be ok.
   
   Yeah, that is the line I saw for 1055. I was running the latest commit on this branch at that time.
   
   > @Manno15 the feedback on logging was great. I made some quick changes with the following goals.
   
   Great, I like the logging changes you made. I will test them out today. 
   
   EDIT: After testing, the logs look great. A lot less chatty on the INFO level. Though, I am not quite sure how to change the log levels for the compactor and coordinator log files. They don't seem to take after the logger settings. 
   
   > @Manno15 re the consistency of those fields, that data was coming from a thrift class and GSON generator json, so not sure they could reliably be ordered consistently.
   
   Ah, that makes sense. It's not a huge deal, just something I noticed while reading through it.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633720108



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       To be more specific, my IDE indicated nothing calls [RunningCompaction.getTserver()](https://github.com/apache/accumulo/blob/cdfec2afccd22ea344ad8fd3a3ce622ee5b87324/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java#L58)




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r638380580



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
##########
@@ -534,10 +540,10 @@ public static AccumuloConfiguration getCompactionConfig(CompactionKind kind, Tab
   }
 
   static StoredTabletFile compact(Tablet tablet, CompactionJob job, Set<StoredTabletFile> jobFiles,
-      Long compactionId, boolean propogateDeletes, CompactableImpl.CompactionHelper helper,
-      List<IteratorSetting> iters, CompactionCheck compactionCheck, RateLimiter readLimiter,
-      RateLimiter writeLimiter, CompactionStats stats)
-      throws IOException, CompactionCanceledException {
+      Long compactionId, Set<StoredTabletFile> selectedFiles, boolean propogateDeletes,

Review comment:
       @DomGarguilo I ran
   
   ```bash
   find . -name "*.java" | xargs sed -i '' -e 's/propogate/propagate/g'
   ```
   
    and pushed the changes in d90f42178f76171e785372e0bf1a04359d24e816




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635582997



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(
+      Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions, Tablet tablet,
+      Map<ExternalCompactionId,String> externalCompactionsToRemove) {
+    CompactionKind extKind = null;
+    boolean unexpectedExternal = false;
+    Set<StoredTabletFile> tmpSelectedFiles = null;
+    Boolean selAll = null;
+    Long cid = null;
+    Boolean propDel = null;
+    int count = 0;
+
+    ArrayList<String> reasons = new ArrayList<>();
+
+    for (Entry<ExternalCompactionId,ExternalCompactionMetadata> entry : extCompactions.entrySet()) {

Review comment:
       I made a comment on #2111 that this code should be refactored so that it could be unit tested.  Thinking it would be  best to wait until then to refactor it.




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   @milleruntime - what's the best way to approach that? Create a branch off of this branch, remove the thrift files from this branch and remove the non-thrift files from the other?


-- 
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.

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



[GitHub] [accumulo] Manno15 edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844219494






-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   > it seems that the way the external compactions are exposed through to the user in the compaction configuration is through the specially named externalQueue, which seems to be a special feature of the default compaction service. Is that correct?
   
   Currently, yes. I believe the intention was to allow for users to configure compactions on their table(s) such that some could run internally (say, smaller ones) and other (larger) externally.
   
   > I'm wondering if there are ways this could be made a bit more modular and the configuration a bit more explicit, so that the default compaction service isn't so monolithic, and users can very explicitly swap out the external executor implementation for another one.
   
   I don't think that one of our goals was to create a pluggable external implementation. It's possible it was, I just don't remember that. Is that something we want?


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   There are two approvals on this. I'm planning on merging this early next week.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633932801



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
##########
@@ -1654,6 +1658,79 @@ public void compact(TInfo tinfo, TCredentials credentials, String lock, String t
     return ret;
   }
 
+  @Override
+  public List<TCompactionQueueSummary> getCompactionQueueInfo(TInfo tinfo, TCredentials credentials)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    return server.getCompactionManager().getCompactionQueueSummaries();
+  }
+
+  @Override
+  public TExternalCompactionJob reserveCompactionJob(TInfo tinfo, TCredentials credentials,
+      String queueName, long priority, String compactor, String externalCompactionId)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    ExternalCompactionId eci = ExternalCompactionId.of(externalCompactionId);
+
+    var extCompaction = server.getCompactionManager().reserveExternalCompaction(queueName, priority,
+        compactor, eci);
+
+    if (extCompaction != null) {
+      return extCompaction.toThrift();
+    }
+
+    return new TExternalCompactionJob();
+  }
+
+  @Override
+  public void compactionJobFinished(TInfo tinfo, TCredentials credentials,
+      String externalCompactionId, TKeyExtent extent, long fileSize, long entries)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    server.getCompactionManager().commitExternalCompaction(
+        ExternalCompactionId.of(externalCompactionId), KeyExtent.fromThrift(extent),
+        server.getOnlineTablets(), fileSize, entries);
+  }
+
+  @Override
+  public void compactionJobFailed(TInfo tinfo, TCredentials credentials,
+      String externalCompactionId, TKeyExtent extent) throws TException {
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    server.getCompactionManager().externalCompactionFailed(
+        ExternalCompactionId.of(externalCompactionId), KeyExtent.fromThrift(extent),
+        server.getOnlineTablets());
+  }
+
+  @Override
+  public boolean isRunningExternalCompaction(TInfo tinfo, TCredentials credentials,

Review comment:
       This thrift method may not be called by 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.

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



[GitHub] [accumulo] milleruntime commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r636093287



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(

Review comment:
       Looks good, 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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632748893



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -539,54 +766,58 @@ public boolean isCompactionEnabled() {
     }
   }
 
-  @Override
-  public void compact(CompactionServiceId service, CompactionJob job, RateLimiter readLimiter,
-      RateLimiter writeLimiter, long queuedTime) {
-
-    Set<StoredTabletFile> jobFiles = job.getFiles().stream()
-        .map(cf -> ((CompactableFileImpl) cf).getStoredTabletFile()).collect(Collectors.toSet());
-
-    Long compactionId = null;
+  private static class CompactionInfo {
+    Set<StoredTabletFile> jobFiles;
     Long checkCompactionId = null;
     boolean propogateDeletes = true;
     CompactionHelper localHelper;
     List<IteratorSetting> iters = List.of();
     CompactionConfig localCompactionCfg;
+    boolean selectedAll;
+    Set<StoredTabletFile> selectedFiles;
+  }
+
+  private CompactionInfo reserveFilesForCompaction(CompactionServiceId service, CompactionJob job) {

Review comment:
       Added a comment and used optional




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   On a test table in Uno w/ 2B entries and 128 tablets I was able to run 40 concurrent external compactions in AKS Kubernetes today.  I am continuing to poke at this, but I think its looking pretty good.  I hope to run test on cluster+Kubernetes soon instead of Uno+Kubernetes.
   
   ```
   root@uno> listcompactions
    SERVER               | AGE       | TYPE  | REASON | READ  | WROTE | TABLE           | TABLET                                   | INPUT | OUTPUT                              | ITERATORS | ITERATOR OPTIONS
   172.20.128.185:9101 (ext) |   26s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9401a;92012                            |     1 |        /2/t-0000cj0/A0002ay3.rf_tmp |        [] | {}
   172.20.128.188:9101 (ext) |  23s856ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;ba01;b801                              |     1 |        /2/t-00017pv/A0002ayb.rf_tmp |        [] | {}
   172.20.128.41:9101 (ext) |  34s598ms |  FULL |   USER | 3.64M | 3.64M |       testTable | 2;0801b;0602                             |     1 |        /2/t-00004hf/A0002axd.rf_tmp |        [] | {}
   172.20.128.53:9101 (ext) |  35s340ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;02014<                                 |     1 |        /2/t-00017dz/A0002axb.rf_tmp |        [] | {}
   172.20.129.116:9101 (ext) |  23s812ms |  FULL |   USER | 2.40M | 2.40M |       testTable | 2;b801;b601                              |     1 |        /2/t-00004ij/A0002ayc.rf_tmp |        [] | {}
   172.20.129.13:9101 (ext) |   22s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b400b;b200b                            |     1 |        /2/t-0000cjc/A0002aye.rf_tmp |        [] | {}
   172.20.129.184:9101 (ext) |  25s695ms |  FULL |   USER | 2.79M | 2.79M |       testTable | 2;9801c;9602                             |     1 |        /2/t-00004gy/A0002ay7.rf_tmp |        [] | {}
   172.20.129.195:9101 (ext) |  34s670ms |  FULL |   USER | 3.30M | 3.30M |       testTable | 2;0401a;02014                            |     1 |        /2/t-0000cl4/A0002axc.rf_tmp |        [] | {}
   172.20.129.80:9101 (ext) |  28s262ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8c015;8a01e                            |     1 |        /2/t-0000cq8/A0002axv.rf_tmp |        [] | {}
   172.20.130.1:9101 (ext) |  24s517ms |  FULL |   USER | 2.51M | 2.51M |       testTable | 2;be00c;bc00e                            |     1 |        /2/t-00017xu/A0002ay9.rf_tmp |        [] | {}
   172.20.130.104:9101 (ext) |  23s609ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b601;b400b                             |     1 |        /2/t-00017s0/A0002ayd.rf_tmp |        [] | {}
   172.20.130.113:9101 (ext) |  30s713ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;4201;4                                 |     1 |        /2/t-00017kz/A0002axn.rf_tmp |        [] | {}
   172.20.130.201:9101 (ext) |  26s766ms |  FULL |   USER | 2.91M | 2.91M |       testTable | 2;9;8e00d                                |     1 |        /2/t-000000d/A0002ay0.rf_tmp |        [] | {}
   172.20.130.249:9101 (ext) |  28s350ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;46017;44014                            |     1 |        /2/t-00017xf/A0002axu.rf_tmp |        [] | {}
   172.20.131.143:9101 (ext) |  24s460ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;bc00e;ba01                             |     1 |        /2/t-0000cnb/A0002aya.rf_tmp |        [] | {}
   172.20.131.150:9101 (ext) |  25s101ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9a01f;9801c                            |     1 |        /2/t-00017qv/A0002ay8.rf_tmp |        [] | {}
   172.20.131.35:9101 (ext) |  26s969ms |  FULL |   USER | 2.77M | 2.77M |       testTable | 2;48016;46017                            |     1 |        /2/t-00004ho/A0002axz.rf_tmp |        [] | {}
   172.20.131.82:9101 (ext) |  33s715ms |  FULL |   USER | 3.36M | 3.36M |       testTable | 2;0c01a;0a01d                            |     1 |        /2/t-0000ctq/A0002axg.rf_tmp |        [] | {}
   172.20.132.14:9101 (ext) |  25s729ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4c01;4a018                             |     1 |        /2/t-0000col/A0002ay6.rf_tmp |        [] | {}
   172.20.132.141:9101 (ext) |  28s416ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8a01e;8801d                            |     1 |        /2/t-00017so/A0002axt.rf_tmp |        [] | {}
   172.20.132.187:9101 (ext) |  28s737ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8801d;86021                            |     1 |        /2/t-00004gp/A0002axs.rf_tmp |        [] | {}
   172.20.132.226:9101 (ext) |  30s934ms |  FULL |   USER | 4.30M | 4.30M |       testTable | 2;8;7e00d                                |     1 |        /2/t-0000000/A0002axl.rf_tmp |        [] | {}
   172.20.132.26:9101 (ext) |  29s994ms |  FULL |   USER | 3.17M | 3.17M |       testTable | 2;22016;2                                |     1 |        /2/t-00017lc/A0002axq.rf_tmp |        [] | {}
   172.20.133.113:9101 (ext) |  29s711ms |  FULL |   USER | 3.20M | 3.20M |       testTable | 2;44014;4201                             |     1 |        /2/t-0000clm/A0002axr.rf_tmp |        [] | {}
   172.20.133.144:9101 (ext) |  32s707ms |  FULL |   USER | 4.10M | 4.10M |       testTable | 2;1;0e013                                |     1 |        /2/t-0000008/A0002axi.rf_tmp |        [] | {}
   172.20.133.182:9101 (ext) |  28s313ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8e00d;8c015                            |     1 |        /2/t-00017zq/A0002axw.rf_tmp |        [] | {}
   172.20.133.255:9101 (ext) |  26s703ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;92012;9                                |     1 |        /2/t-00017kk/A0002ay1.rf_tmp |        [] | {}
   172.20.133.55:9101 (ext) |  25s724ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;26024;2401d                            |     1 |        /2/t-00017us/A0002ay5.rf_tmp |        [] | {}
   172.20.134.114:9101 (ext) |  26s191ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4a018;48016                            |     1 |        /2/t-00017wj/A0002ay2.rf_tmp |        [] | {}
   172.20.134.139:9101 (ext) |  25s993ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9602;9401a                             |     1 |        /2/t-00017rl/A0002ay4.rf_tmp |        [] | {}
   172.20.134.161:9101 (ext) |  30s202ms |  FULL |   USER | 3.70M | 3.70M |       testTable | 2;86021;84019                            |     1 |        /2/t-00017hh/A0002axp.rf_tmp |        [] | {}
   172.20.134.250:9101 (ext) |  30s186ms |  FULL |   USER | 3.19M | 3.19M |       testTable | 2;84019;82013                            |     1 |        /2/t-0000co6/A0002axo.rf_tmp |        [] | {}
   172.20.134.61:9101 (ext) |   34s46ms |  FULL |   USER | 4.52M | 4.52M |       testTable | 2;0602;0401a                             |     1 |        /2/t-00017mi/A0002axf.rf_tmp |        [] | {}
   172.20.135.166:9101 (ext) |   33s46ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;0e013;0c01a                            |     1 |        /2/t-0001816/A0002axh.rf_tmp |        [] | {}
   172.20.135.191:9101 (ext) |  32s423ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2;1e00e                                |     1 |        /2/t-0000005/A0002axj.rf_tmp |        [] | {}
   172.20.135.222:9101 (ext) |  34s165ms |  FULL |   USER | 4.43M | 4.43M |       testTable | 2;0a01d;0801b                            |     1 |        /2/t-00017gs/A0002axe.rf_tmp |        [] | {}
   172.20.135.41:9101 (ext) |  26s997ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2401d;22016                            |     1 |        /2/t-0000cle/A0002axy.rf_tmp |        [] | {}
   172.20.136.109:9101 (ext) |  27s177ms |  FULL |   USER | 2.98M | 2.98M |       testTable | 2;1201;1                                 |     1 |        /2/t-00017v7/A0002axx.rf_tmp |        [] | {}
   172.20.136.162:9101 (ext) |  31s305ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4;3e00f                                |     1 |        /2/t-0000002/A0002axk.rf_tmp |        [] | {}
   172.20.136.52:9101 (ext) |  30s742ms |  FULL |   USER | 3.35M | 3.35M |       testTable | 2;82013;8                                |     1 |        /2/t-00017iu/A0002axm.rf_tmp |        [] | {}
   ```


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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






-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r631956918



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
##########
@@ -204,6 +210,54 @@ public boolean testClassLoad(final String className, final String asTypeName)
     }
   }
 
+  @Override
+  public List<ActiveCompaction> getActiveCompactions()
+      throws AccumuloException, AccumuloSecurityException {
+
+    List<HostAndPort> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    List<String> tservers = getTabletServers();
+
+    int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
+    var executorService =
+        ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions", false);
+    try {
+      List<Future<List<ActiveCompaction>>> futures = new ArrayList<>();
+
+      for (String tserver : tservers) {
+        futures.add(executorService.submit(() -> getActiveCompactions(tserver)));
+      }
+
+      for (HostAndPort compactorAddr : compactors) {
+        futures
+            .add(
+                executorService
+                    .submit(
+                        () -> ExternalCompactionUtil.getActiveCompaction(compactorAddr, context)
+                            .stream().map(tac -> new ActiveCompactionImpl(context, tac,
+                                compactorAddr, CompactionHost.Type.COMPACTOR))
+                            .collect(Collectors.toList())));

Review comment:
       oh yeah this is awful, i think I did this.  I suspect breaking it out to multiple lines would help a lot.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632595852



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       Removing `thrift` from the coordinator property names would make them consistent w/ the compactor properties.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633713455



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       I may be missing something.  I was looking at the code saw a good bit of effort went into finding `tsi`, however it seems after the information was found that it was never 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.

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



[GitHub] [accumulo] ctubbsii commented on pull request #2096: External Compactions

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


   @keith-turner wrote:
   > These RPCs are structured specifically for the way the rest of the distributed system (coordinator and compactors) works. Should these tserver RPCs be generalized? If so, how?
   
   Sorry, I don't know enough about the current implementation to understand the question yet, in order to answer it. The generalized interface I was thinking about was something like `BiFunction<Set<File>, CompactionConfig, Future<File>>`. I was thinking any necessary RPCs would be implementation-specific, and independent of Accumulo's RPCs. My preconceptions of what this would look like have been broken, so I would need to do some catch up understanding before I could have specific thoughts on how they could be improved.
   
   > > My understanding of the original idea to create an "external" compactions, was to create them "external" to Accumulo... something Accumulo could "submit" work to do and not care about the implementation, and get back a new compacted file.
   > 
   > That was never an explicit goal I had.
   
   Okay, I guess I just had a different idea in mind of what "external" meant.
   
   > Is there another path other than what I have described? Does this path actually lower the complexity? I am not sure it would, I think the answer to that depends on what these new API/SPI calls are.
   
   If we went with the `BiFunction<Set<File>, CompactionConfig, Future<File>>` concept, or similar, then nothing internal need be exposed to it. The RFile config settings (locality groups, etc.) and iterators are already "public API". A lot of the compaction complexity, with the services, planners, executors, queues, workers, etc. could sit inside an implementation. Or, all that could be left as-is, and just expect the pluggable part to be the "last stage" after going through that management pipeline, which I think is basically how it is now... but not expressed as simply as I have with the "BiFunction" expression.
   
   I'm curious, do the external compactors support RFile encryption and other RFile-specific configs (like block sizes, locality groups, etc.) today? My guess is that they do, because it's still all the same Accumulo code, just running in a different process, but just curious how much you've exercised those while testing 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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632595852



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       Removing `thrift` from the property names would make them consistent w/ the compactor properties.




-- 
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.

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



[GitHub] [accumulo] DomGarguilo commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635326117



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
##########
@@ -534,10 +540,10 @@ public static AccumuloConfiguration getCompactionConfig(CompactionKind kind, Tab
   }
 
   static StoredTabletFile compact(Tablet tablet, CompactionJob job, Set<StoredTabletFile> jobFiles,
-      Long compactionId, boolean propogateDeletes, CompactableImpl.CompactionHelper helper,
-      List<IteratorSetting> iters, CompactionCheck compactionCheck, RateLimiter readLimiter,
-      RateLimiter writeLimiter, CompactionStats stats)
-      throws IOException, CompactionCanceledException {
+      Long compactionId, Set<StoredTabletFile> selectedFiles, boolean propogateDeletes,

Review comment:
       Looks like there are lots of occurrences of this typo actually. propogate vs propagate. Not sure if this is the correct place to fix this or not.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r638372331



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -0,0 +1,898 @@
+/*
+ * 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.compactor;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Client;
+import org.apache.accumulo.core.compaction.thrift.CompactorService;
+import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
+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.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.trace.TraceUtil;
+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.ServerServices;
+import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.core.util.threads.Threads;
+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.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.ServerOpts;
+import org.apache.accumulo.server.compaction.CompactionInfo;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.hadoop.fs.Path;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.google.common.base.Preconditions;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class Compactor extends AbstractServer implements CompactorService.Iface {
+
+  public static class CompactorServerOpts extends ServerOpts {
+    @Parameter(required = true, names = {"-q", "--queue"}, description = "compaction queue name")
+    private String queueName = null;
+
+    public String getQueueName() {
+      return queueName;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(Compactor.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long TIME_BETWEEN_CANCEL_CHECKS = 5 * 60 * 1000;
+
+  private static final long TEN_MEGABYTES = 10485760;
+  private static final CompactionCoordinatorService.Client.Factory COORDINATOR_CLIENT_FACTORY =
+      new CompactionCoordinatorService.Client.Factory();
+
+  protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  private final UUID compactorId = UUID.randomUUID();
+  private final AccumuloConfiguration aconf;
+  private final String queueName;
+  protected final AtomicReference<ExternalCompactionId> currentCompactionId =
+      new AtomicReference<>();
+
+  private SecurityOperation security;
+  private ServiceLock compactorLock;
+  private ServerAddress compactorAddress = null;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private final AtomicBoolean compactionRunning = new AtomicBoolean(false);
+
+  protected Compactor(CompactorServerOpts opts, String[] args) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = getConfiguration();
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected Compactor(CompactorServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = conf;
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor,
+      long timeBetweenChecks) {
+    schedExecutor.scheduleWithFixedDelay(() -> checkIfCanceled(), 0, timeBetweenChecks,
+        TimeUnit.MILLISECONDS);
+  }
+
+  protected void checkIfCanceled() {
+    TExternalCompactionJob job = JOB_HOLDER.getJob();
+    if (job != null) {
+      try {
+        var extent = KeyExtent.fromThrift(job.getExtent());
+        var ecid = ExternalCompactionId.of(job.getExternalCompactionId());
+
+        TabletMetadata tabletMeta =
+            getContext().getAmple().readTablet(extent, ColumnType.ECOMP, ColumnType.PREV_ROW);
+        if (tabletMeta == null || !tabletMeta.getExtent().equals(extent)
+            || !tabletMeta.getExternalCompactions().containsKey(ecid)) {
+          // table was deleted OR tablet was split or merged OR tablet no longer thinks compaction
+          // is running for some reason
+          LOG.info("Cancelling compaction {} that no longer has a metadata entry at {}", ecid,
+              extent);
+          JOB_HOLDER.cancel(job.getExternalCompactionId());
+          return;
+        }
+
+        if (job.getKind() == TCompactionKind.USER) {
+          String zTablePath = Constants.ZROOT + "/" + getContext().getInstanceID()
+              + Constants.ZTABLES + "/" + extent.tableId() + Constants.ZTABLE_COMPACT_CANCEL_ID;
+          byte[] id = getContext().getZooCache().get(zTablePath);
+          if (id == null) {
+            // table probably deleted
+            LOG.info("Cancelling compaction {} for table that no longer exists {}", ecid, extent);
+            JOB_HOLDER.cancel(job.getExternalCompactionId());
+            return;
+          } else {
+            var cancelId = Long.parseLong(new String(id, UTF_8));
+
+            if (cancelId >= job.getUserCompactionId()) {
+              LOG.info("Cancelling compaction {} because user compaction was canceled");
+              JOB_HOLDER.cancel(job.getExternalCompactionId());
+              return;
+            }
+          }
+        }
+      } catch (RuntimeException e) {
+        LOG.warn("Failed to check if compaction {} for {} was canceled.",
+            job.getExternalCompactionId(), KeyExtent.fromThrift(job.getExtent()), e);
+      }
+    }
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   *
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void announceExistence(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+
+    String hostPort = ExternalCompactionUtil.getHostPortString(clientAddress);
+
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    String compactorQueuePath =
+        getContext().getZooKeeperRoot() + Constants.ZCOMPACTORS + "/" + this.queueName;
+    String zPath = compactorQueuePath + "/" + hostPort;
+
+    try {
+      zoo.mkdirs(compactorQueuePath);
+      zoo.putPersistentData(zPath, 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;
+    }
+
+    compactorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+        ServiceLock.path(zPath), compactorId);
+    LockWatcher lw = new LockWatcher() {
+      @Override
+      public void lostLock(final LockLossReason reason) {
+        Halt.halt(1, () -> {
+          LOG.error("Compactor lost lock (reason = {}), exiting.", reason);
+          gcLogger.logGCInfo(getConfiguration());
+        });
+      }
+
+      @Override
+      public void unableToMonitorLockNode(final Exception e) {
+        Halt.halt(1, () -> LOG.error("Lost ability to monitor Compactor lock, exiting.", e));
+      }
+    };
+
+    try {
+      byte[] lockContent =
+          new ServerServices(hostPort, Service.COMPACTOR_CLIENT).toString().getBytes(UTF_8);
+      for (int i = 0; i < 25; i++) {
+        zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP);
+
+        if (compactorLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained Compactor lock {}", compactorLock.getLockPath());
+          return;
+        }
+        LOG.info("Waiting for Compactor 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 tablet server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Start this Compactors thrift service to handle incoming client requests
+   *
+   * @return address of this compactor client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCompactorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, getClass(), getConfiguration());
+    }
+    final CompactorService.Processor<Iface> processor = new CompactorService.Processor<>(rpcProxy);
+    Property maxMessageSizeProperty = (aconf.get(Property.COMPACTOR_MAX_MESSAGE_SIZE) != null
+        ? Property.COMPACTOR_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COMPACTOR_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COMPACTOR_PORTSEARCH, Property.COMPACTOR_MINTHREADS,
+        Property.COMPACTOR_MINTHREADS_TIMEOUT, Property.COMPACTOR_THREADCHECK,
+        maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  /**
+   * Called by a CompactionCoordinator to cancel the currently running compaction
+   *
+   * @param tinfo
+   *          trace info
+   * @param credentials
+   *          caller credentials
+   * @param externalCompactionId
+   *          compaction id
+   * @throws UnknownCompactionIdException
+   *           if the externalCompactionId does not match the currently executing compaction
+   */
+  @Override
+  public void cancel(TInfo tinfo, TCredentials credentials, String externalCompactionId)
+      throws TException {
+    // do not expect users to call this directly, expect other tservers to call this method
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+    cancel(externalCompactionId);
+  }
+
+  /**
+   * Cancel the compaction with this id.
+   *
+   * @param externalCompactionId
+   *          compaction id
+   * @throws UnknownCompactionIdException
+   *           if the externalCompactionId does not match the currently executing compaction
+   * @throws TException
+   *           thrift error
+   */
+  private void cancel(String externalCompactionId) throws TException {
+    if (JOB_HOLDER.cancel(externalCompactionId)) {
+      LOG.info("Cancel requested for compaction job {}", externalCompactionId);
+    } else {
+      throw new UnknownCompactionIdException();
+    }
+  }
+
+  /**
+   * Send an update to the CompactionCoordinator for this job
+   *
+   * @param job
+   *          compactionJob
+   * @param state
+   *          updated state
+   * @param message
+   *          updated message
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected void updateCompactionState(TExternalCompactionJob job, TCompactionState state,
+      String message) throws RetriesExceededException {
+    RetryableThriftCall<String> thriftCall = new RetryableThriftCall<>(1000,
+        RetryableThriftCall.MAX_WAIT_TIME, 25, new RetryableThriftFunction<String>() {
+          @Override
+          public String execute() throws TException {
+            Client coordinatorClient = getCoordinatorClient();
+            try {
+              coordinatorClient.updateCompactionStatus(TraceUtil.traceInfo(),
+                  getContext().rpcCreds(), job.getExternalCompactionId(), state, message,
+                  System.currentTimeMillis());
+              return "";
+            } finally {
+              ThriftUtil.returnClient(coordinatorClient);
+            }
+          }
+        });
+    thriftCall.run();
+  }
+
+  /**
+   * Notify the CompactionCoordinator the job failed
+   *
+   * @param job
+   *          current compaction job
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected void updateCompactionFailed(TExternalCompactionJob job)
+      throws RetriesExceededException {
+    RetryableThriftCall<String> thriftCall = new RetryableThriftCall<>(1000,
+        RetryableThriftCall.MAX_WAIT_TIME, 25, new RetryableThriftFunction<String>() {
+          @Override
+          public String execute() throws TException {
+            Client coordinatorClient = getCoordinatorClient();
+            try {
+              coordinatorClient.compactionFailed(TraceUtil.traceInfo(), getContext().rpcCreds(),
+                  job.getExternalCompactionId(), job.extent);
+              return "";
+            } finally {
+              ThriftUtil.returnClient(coordinatorClient);
+            }
+          }
+        });
+    thriftCall.run();
+  }
+
+  /**
+   * Update the CompactionCoordinator with the stats from the completed job
+   *
+   * @param job
+   *          current compaction job
+   * @param stats
+   *          compaction stats
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats)
+      throws RetriesExceededException {
+    RetryableThriftCall<String> thriftCall = new RetryableThriftCall<>(1000,
+        RetryableThriftCall.MAX_WAIT_TIME, 25, new RetryableThriftFunction<String>() {
+          @Override
+          public String execute() throws TException {
+            Client coordinatorClient = getCoordinatorClient();
+            try {
+              coordinatorClient.compactionCompleted(TraceUtil.traceInfo(), getContext().rpcCreds(),
+                  job.getExternalCompactionId(), job.extent, stats);
+              return "";
+            } finally {
+              ThriftUtil.returnClient(coordinatorClient);
+            }
+          }
+        });
+    thriftCall.run();
+  }
+
+  /**
+   * Get the next job to run
+   *
+   * @param uuid
+   *          uuid supplier
+   * @return CompactionJob
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected TExternalCompactionJob getNextJob(Supplier<UUID> uuid) throws RetriesExceededException {
+    RetryableThriftCall<TExternalCompactionJob> nextJobThriftCall =
+        new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 0,
+            new RetryableThriftFunction<TExternalCompactionJob>() {
+              @Override
+              public TExternalCompactionJob execute() throws TException {
+                Client coordinatorClient = getCoordinatorClient();
+                try {
+                  ExternalCompactionId eci = ExternalCompactionId.generate(uuid.get());
+                  LOG.trace("Attempting to get next job, eci = {}", eci);
+                  currentCompactionId.set(eci);
+                  return coordinatorClient.getCompactionJob(TraceUtil.traceInfo(),
+                      getContext().rpcCreds(), queueName,
+                      ExternalCompactionUtil.getHostPortString(compactorAddress.getAddress()),
+                      eci.toString());
+                } catch (Exception e) {
+                  currentCompactionId.set(null);
+                  throw e;
+                } finally {
+                  ThriftUtil.returnClient(coordinatorClient);
+                }
+              }
+            });
+    return nextJobThriftCall.run();
+  }
+
+  /**
+   * Get the client to the CompactionCoordinator
+   *
+   * @return compaction coordinator client
+   * @throws TTransportException
+   *           when unable to get client
+   */
+  protected CompactionCoordinatorService.Client getCoordinatorClient() throws TTransportException {
+    HostAndPort coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(getContext());
+    if (null == coordinatorHost) {
+      throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper");
+    }
+    LOG.trace("CompactionCoordinator address is: {}", coordinatorHost);
+    return ThriftUtil.getClient(COORDINATOR_CLIENT_FACTORY, coordinatorHost, getContext());
+  }
+
+  /**
+   * Create compaction runnable
+   *
+   * @param job
+   *          compaction job
+   * @param totalInputEntries
+   *          object to capture total entries
+   * @param totalInputBytes
+   *          object to capture input file size
+   * @param started
+   *          started latch
+   * @param stopped
+   *          stopped latch
+   * @param err
+   *          reference to error
+   * @return Runnable compaction job
+   */
+  protected Runnable createCompactionJob(final TExternalCompactionJob job,
+      final LongAdder totalInputEntries, final LongAdder totalInputBytes,
+      final CountDownLatch started, final CountDownLatch stopped,
+      final AtomicReference<Throwable> err) {
+
+    return new Runnable() {
+      @Override
+      public void run() {
+        // Its only expected that a single compaction runs at a time. Multiple compactions running
+        // at a time could cause odd behavior like out of order and unexpected thrift calls to the
+        // coordinator. This is a sanity check to ensure the expectation is met. Should this check
+        // ever fail, it means there is a bug elsewhere.
+        Preconditions.checkState(compactionRunning.compareAndSet(false, true));
+        try {
+          LOG.info("Starting up compaction runnable for job: {}", job);
+          updateCompactionState(job, TCompactionState.STARTED, "Compaction started");
+
+          final AccumuloConfiguration tConfig;
+
+          if (!job.getTableCompactionProperties().isEmpty()) {
+            tConfig = new ConfigurationCopy(DefaultConfiguration.getInstance());

Review comment:
       Resolved this in c2da878084c51d59acf6cbda9a92f59d629d6a2b




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   Omitting a lot of detail, the following is basics of what the current system enable.  The first step uses a pluggable planner.
   
   1. Plan compaction :  Take current tablets files, current running compactions for the tablet, and emit set of compactions jobs.  A compaction job is set of files to compact, a  priority, and a destination queue.
    2. Queue Jobs : An attempt to queue the jobs emitted from the planner is made by doing the following.
       1. Cancel anything that was previously queued by the planner for the tablet.  If this fails, go back to planning step and try again because things probably changed during planning.
       2. Queue the new job from the planner on the desired priority queues
    3. When an internal or external compactor thread finishes a task, it takes the next highest priority job from the queue and actually run the compaction.
   
   I don' think interface `BiFunction<Set<File>, CompactionConfig, Future<File>>` would be sufficient to achieve this functionality.  It would not support the current functionality of priority queues and canceling queued work when things change (new files arrive, job start/finish during planning). In the current impl the priority queue for local compactions is very precise.  For external compaction there is essentially a global priority queue that is approximate (eventually consistent) that may not always start the highest priority job next, but usually will.
   
   


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   Thinking through plugin interfaces and alternative designs, I thought of a very different way compactions could work in Accumulo.  We could have a process (like the manager or compaction coordinator) that continually scans the files in the metadata table looking for compaction work that needs to be done per tablet and gives that to external compaction processes.  Under this model tablet server would have no compaction code at all and there would only be external compactions. The tserver would only need a mechanism to replace a set of files in a tablet to support this model.  I think this is an interesting model to think about, but I am not advocating for it.  It has pluses and minuses over the current model.  The reason I am sharing it is that I think it informs the general discussion about API and SPI interfaces around compaction and supporting alternative designs.


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634270225



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
##########
@@ -1654,6 +1658,79 @@ public void compact(TInfo tinfo, TCredentials credentials, String lock, String t
     return ret;
   }
 
+  @Override
+  public List<TCompactionQueueSummary> getCompactionQueueInfo(TInfo tinfo, TCredentials credentials)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    return server.getCompactionManager().getCompactionQueueSummaries();
+  }
+
+  @Override
+  public TExternalCompactionJob reserveCompactionJob(TInfo tinfo, TCredentials credentials,
+      String queueName, long priority, String compactor, String externalCompactionId)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    ExternalCompactionId eci = ExternalCompactionId.of(externalCompactionId);
+
+    var extCompaction = server.getCompactionManager().reserveExternalCompaction(queueName, priority,
+        compactor, eci);
+
+    if (extCompaction != null) {
+      return extCompaction.toThrift();
+    }
+
+    return new TExternalCompactionJob();
+  }
+
+  @Override
+  public void compactionJobFinished(TInfo tinfo, TCredentials credentials,
+      String externalCompactionId, TKeyExtent extent, long fileSize, long entries)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    server.getCompactionManager().commitExternalCompaction(
+        ExternalCompactionId.of(externalCompactionId), KeyExtent.fromThrift(extent),
+        server.getOnlineTablets(), fileSize, entries);
+  }
+
+  @Override
+  public void compactionJobFailed(TInfo tinfo, TCredentials credentials,
+      String externalCompactionId, TKeyExtent extent) throws TException {
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    server.getCompactionManager().externalCompactionFailed(
+        ExternalCompactionId.of(externalCompactionId), KeyExtent.fromThrift(extent),
+        server.getOnlineTablets());
+  }
+
+  @Override
+  public boolean isRunningExternalCompaction(TInfo tinfo, TCredentials credentials,

Review comment:
       I'll 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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   Another option would be to click the "Viewed" checkbox on all of the thrift files, then they would be removed from your view. Not sure if that persists across browser sessions or not.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632738659



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(

Review comment:
       I will add some comments documenting what is going on. Also I opened #2111 because I am not sure this code is tested.




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634329118



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =

Review comment:
       @keith-turner - I resolved this in f39707a492aea982b5127c7358e4a7f209a812f0




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634739521



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }
+
+          // As a fallback, try them all
+          if (!matchFound) {
+            LOG.debug("Checking all tservers for external running compaction, extent: {}", extent);
+            for (TServerInstance tsi : tservers) {
+              TabletClientService.Client client = null;
+              try {
+                client = getTabletServerConnection(tsi);
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                }
+              } catch (TException e) {
+                LOG.error(
+                    "Error from tserver {} while trying to check if external compaction is running, trying next tserver",
+                    ExternalCompactionUtil.getHostPortString(tsi.getHostAndPort()), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            }
+          }
+
+          if (!matchFound) {
+            LOG.warn(
+                "There is an external compaction running on a compactor, but could not find corresponding tablet server. Extent: {}, Compactor: {}, Compaction: {}",
+                extent, hp, job);
+          }
+        });
+      }
+      tservers.clear();
+    } else {
+      LOG.info("No running tablet servers found, continuing startup");
+    }
+
+    tserverSet.startListeningForTabletServerChanges();
+    startDeadCompactionDetector();
+
+    LOG.info("Starting loop to check tservers for compaction summaries");
+    while (!shutdown) {
+      long start = System.currentTimeMillis();
+      tserverSet.getCurrentServers().forEach(tsi -> {
+        try {
+          TabletClientService.Client client = null;
+          try {
+            LOG.debug("Contacting tablet server {} to get external compaction summaries",
+                tsi.getHostPort());
+            client = getTabletServerConnection(tsi);
+            List<TCompactionQueueSummary> summaries =
+                client.getCompactionQueueInfo(TraceUtil.traceInfo(), getContext().rpcCreds());
+            summaries.forEach(summary -> {
+              QueueAndPriority qp =
+                  QueueAndPriority.get(summary.getQueue().intern(), summary.getPriority());
+              synchronized (qp) {
+                TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(qp.getQueue(), k -> 0L);
+                QUEUE_SUMMARIES.update(tsi, summaries);
+              }
+            });
+          } finally {
+            ThriftUtil.returnClient(client);
+          }

Review comment:
       Done in b59c13ff80da6491a89757a00c0fa32166d038d2




-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   @dlmarion it would help a lot for developers reviewing to split this PR up into 2 commits: 1 with your changes and the other with the generated thrift files. This is something we have done in the past with big PRs that have thrift 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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842394039






-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633719112



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       To be more specific, my IDE indicated nothing calls [RunningCompaction.getTserver()](https://github.com/apache/accumulo/blob/cdfec2afccd22ea344ad8fd3a3ce622ee5b87324/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java#L58)




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633619757



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       I guess the question is whether or not the CompactionCoordinator can get stale data from the metadata table. The code tries to find the TabletServer running this compaction. It's possible that the Tablet could be in the process of being re-hosted (in the case of a TServer crash). In the current implementation an entry would not be made in the internal running list because we didn't find the TServer. If we just use the Metadata entries, we could put incorrect information in the running list.




-- 
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.

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



[GitHub] [accumulo] milleruntime commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r636076932



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(
+      Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions, Tablet tablet,
+      Map<ExternalCompactionId,String> externalCompactionsToRemove) {
+    CompactionKind extKind = null;
+    boolean unexpectedExternal = false;
+    Set<StoredTabletFile> tmpSelectedFiles = null;
+    Boolean selAll = null;
+    Long cid = null;
+    Boolean propDel = null;
+    int count = 0;
+
+    ArrayList<String> reasons = new ArrayList<>();
+
+    for (Entry<ExternalCompactionId,ExternalCompactionMetadata> entry : extCompactions.entrySet()) {

Review comment:
       A unit test would be even better! I could give that task a try, if you want some help.




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   That's in a finally block, line 1055 is:
   ```
   Preconditions.checkState(externalCompactionsCommitting.remove(extCompactionId));
   ```


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635606092



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java
##########
@@ -40,14 +42,25 @@
   private final Set<CompactableFile> files;
   private final CompactionKind kind;
   private boolean selectedAll;
+  private boolean hasSelectedAll;
 
-  CompactionJobImpl(long priority, CompactionExecutorId executor, Collection<CompactableFile> files,
-      CompactionKind kind, boolean selectedAllFiles) {
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind, boolean selectedAllFiles) {
     this.priority = priority;
     this.executor = Objects.requireNonNull(executor);
     this.files = Set.copyOf(files);
-    this.kind = kind;
+    this.kind = Objects.requireNonNull(kind);
     this.selectedAll = selectedAllFiles;
+    this.hasSelectedAll = true;
+  }
+
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind) {
+    this.priority = priority;
+    this.executor = Objects.requireNonNull(executor);
+    this.files = Set.copyOf(files);
+    this.kind = Objects.requireNonNull(kind);
+    this.hasSelectedAll = false;

Review comment:
       @milleruntime that was confusing. I made some changes in 2ba605cd62ea4c1d2cc4733c7a562a8f0fb2499d for this, let me know what you think.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   @Manno15 re the exception you saw.  I am seeing slightly different code that @dlmarion  saw at line 1055.  What commit were you testing with?  Or can you look in the source you built from and see what is at CompactableImpl.java line 1055? I am seeing the following line in the latest commit.
   
   ```java
   Preconditions.checkState(!closed);
   ```


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   @Manno15 I think I may have fixed the issue you were seeing in 3095d3d, if line 1055 was the !closed check.  That was an incorrect sanity check, looking at how the close() method works, closed could be true and that would be ok.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633713455



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       I may be missing something.  I was looking at the code saw a good bit of effort went into finding `tsi`, however it seems after the information was found that it was never used.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       To be more specific, my IDE indicated nothing calls [RunningCompaction.getTserver()](https://github.com/apache/accumulo/blob/cdfec2afccd22ea344ad8fd3a3ce622ee5b87324/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java#L58)

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       To be more specific, my IDE indicated nothing calls [RunningCompaction.getTserver()](https://github.com/apache/accumulo/blob/cdfec2afccd22ea344ad8fd3a3ce622ee5b87324/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java#L58)

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       To be more specific, my IDE indicated nothing calls [RunningCompaction.getTserver()](https://github.com/apache/accumulo/blob/cdfec2afccd22ea344ad8fd3a3ce622ee5b87324/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java#L58)




-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-849952173


   On a test table in Uno w/ 2B entries and 128 tablets I was able to run 40 concurrent external compactions in AKS Kubernetes today (had lots of help from @dlmarion  w/ Kubernetes).  I am continuing to poke at this, but I think its looking pretty good.  I hope to run test on cluster+Kubernetes soon instead of Uno+Kubernetes.
   
   ```
   root@uno> listcompactions
    SERVER               | AGE       | TYPE  | REASON | READ  | WROTE | TABLE           | TABLET                                   | INPUT | OUTPUT                              | ITERATORS | ITERATOR OPTIONS
   172.20.128.185:9101 (ext) |   26s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9401a;92012                            |     1 |        /2/t-0000cj0/A0002ay3.rf_tmp |        [] | {}
   172.20.128.188:9101 (ext) |  23s856ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;ba01;b801                              |     1 |        /2/t-00017pv/A0002ayb.rf_tmp |        [] | {}
   172.20.128.41:9101 (ext) |  34s598ms |  FULL |   USER | 3.64M | 3.64M |       testTable | 2;0801b;0602                             |     1 |        /2/t-00004hf/A0002axd.rf_tmp |        [] | {}
   172.20.128.53:9101 (ext) |  35s340ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;02014<                                 |     1 |        /2/t-00017dz/A0002axb.rf_tmp |        [] | {}
   172.20.129.116:9101 (ext) |  23s812ms |  FULL |   USER | 2.40M | 2.40M |       testTable | 2;b801;b601                              |     1 |        /2/t-00004ij/A0002ayc.rf_tmp |        [] | {}
   172.20.129.13:9101 (ext) |   22s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b400b;b200b                            |     1 |        /2/t-0000cjc/A0002aye.rf_tmp |        [] | {}
   172.20.129.184:9101 (ext) |  25s695ms |  FULL |   USER | 2.79M | 2.79M |       testTable | 2;9801c;9602                             |     1 |        /2/t-00004gy/A0002ay7.rf_tmp |        [] | {}
   172.20.129.195:9101 (ext) |  34s670ms |  FULL |   USER | 3.30M | 3.30M |       testTable | 2;0401a;02014                            |     1 |        /2/t-0000cl4/A0002axc.rf_tmp |        [] | {}
   172.20.129.80:9101 (ext) |  28s262ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8c015;8a01e                            |     1 |        /2/t-0000cq8/A0002axv.rf_tmp |        [] | {}
   172.20.130.1:9101 (ext) |  24s517ms |  FULL |   USER | 2.51M | 2.51M |       testTable | 2;be00c;bc00e                            |     1 |        /2/t-00017xu/A0002ay9.rf_tmp |        [] | {}
   172.20.130.104:9101 (ext) |  23s609ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b601;b400b                             |     1 |        /2/t-00017s0/A0002ayd.rf_tmp |        [] | {}
   172.20.130.113:9101 (ext) |  30s713ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;4201;4                                 |     1 |        /2/t-00017kz/A0002axn.rf_tmp |        [] | {}
   172.20.130.201:9101 (ext) |  26s766ms |  FULL |   USER | 2.91M | 2.91M |       testTable | 2;9;8e00d                                |     1 |        /2/t-000000d/A0002ay0.rf_tmp |        [] | {}
   172.20.130.249:9101 (ext) |  28s350ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;46017;44014                            |     1 |        /2/t-00017xf/A0002axu.rf_tmp |        [] | {}
   172.20.131.143:9101 (ext) |  24s460ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;bc00e;ba01                             |     1 |        /2/t-0000cnb/A0002aya.rf_tmp |        [] | {}
   172.20.131.150:9101 (ext) |  25s101ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9a01f;9801c                            |     1 |        /2/t-00017qv/A0002ay8.rf_tmp |        [] | {}
   172.20.131.35:9101 (ext) |  26s969ms |  FULL |   USER | 2.77M | 2.77M |       testTable | 2;48016;46017                            |     1 |        /2/t-00004ho/A0002axz.rf_tmp |        [] | {}
   172.20.131.82:9101 (ext) |  33s715ms |  FULL |   USER | 3.36M | 3.36M |       testTable | 2;0c01a;0a01d                            |     1 |        /2/t-0000ctq/A0002axg.rf_tmp |        [] | {}
   172.20.132.14:9101 (ext) |  25s729ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4c01;4a018                             |     1 |        /2/t-0000col/A0002ay6.rf_tmp |        [] | {}
   172.20.132.141:9101 (ext) |  28s416ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8a01e;8801d                            |     1 |        /2/t-00017so/A0002axt.rf_tmp |        [] | {}
   172.20.132.187:9101 (ext) |  28s737ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8801d;86021                            |     1 |        /2/t-00004gp/A0002axs.rf_tmp |        [] | {}
   172.20.132.226:9101 (ext) |  30s934ms |  FULL |   USER | 4.30M | 4.30M |       testTable | 2;8;7e00d                                |     1 |        /2/t-0000000/A0002axl.rf_tmp |        [] | {}
   172.20.132.26:9101 (ext) |  29s994ms |  FULL |   USER | 3.17M | 3.17M |       testTable | 2;22016;2                                |     1 |        /2/t-00017lc/A0002axq.rf_tmp |        [] | {}
   172.20.133.113:9101 (ext) |  29s711ms |  FULL |   USER | 3.20M | 3.20M |       testTable | 2;44014;4201                             |     1 |        /2/t-0000clm/A0002axr.rf_tmp |        [] | {}
   172.20.133.144:9101 (ext) |  32s707ms |  FULL |   USER | 4.10M | 4.10M |       testTable | 2;1;0e013                                |     1 |        /2/t-0000008/A0002axi.rf_tmp |        [] | {}
   172.20.133.182:9101 (ext) |  28s313ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8e00d;8c015                            |     1 |        /2/t-00017zq/A0002axw.rf_tmp |        [] | {}
   172.20.133.255:9101 (ext) |  26s703ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;92012;9                                |     1 |        /2/t-00017kk/A0002ay1.rf_tmp |        [] | {}
   172.20.133.55:9101 (ext) |  25s724ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;26024;2401d                            |     1 |        /2/t-00017us/A0002ay5.rf_tmp |        [] | {}
   172.20.134.114:9101 (ext) |  26s191ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4a018;48016                            |     1 |        /2/t-00017wj/A0002ay2.rf_tmp |        [] | {}
   172.20.134.139:9101 (ext) |  25s993ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9602;9401a                             |     1 |        /2/t-00017rl/A0002ay4.rf_tmp |        [] | {}
   172.20.134.161:9101 (ext) |  30s202ms |  FULL |   USER | 3.70M | 3.70M |       testTable | 2;86021;84019                            |     1 |        /2/t-00017hh/A0002axp.rf_tmp |        [] | {}
   172.20.134.250:9101 (ext) |  30s186ms |  FULL |   USER | 3.19M | 3.19M |       testTable | 2;84019;82013                            |     1 |        /2/t-0000co6/A0002axo.rf_tmp |        [] | {}
   172.20.134.61:9101 (ext) |   34s46ms |  FULL |   USER | 4.52M | 4.52M |       testTable | 2;0602;0401a                             |     1 |        /2/t-00017mi/A0002axf.rf_tmp |        [] | {}
   172.20.135.166:9101 (ext) |   33s46ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;0e013;0c01a                            |     1 |        /2/t-0001816/A0002axh.rf_tmp |        [] | {}
   172.20.135.191:9101 (ext) |  32s423ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2;1e00e                                |     1 |        /2/t-0000005/A0002axj.rf_tmp |        [] | {}
   172.20.135.222:9101 (ext) |  34s165ms |  FULL |   USER | 4.43M | 4.43M |       testTable | 2;0a01d;0801b                            |     1 |        /2/t-00017gs/A0002axe.rf_tmp |        [] | {}
   172.20.135.41:9101 (ext) |  26s997ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2401d;22016                            |     1 |        /2/t-0000cle/A0002axy.rf_tmp |        [] | {}
   172.20.136.109:9101 (ext) |  27s177ms |  FULL |   USER | 2.98M | 2.98M |       testTable | 2;1201;1                                 |     1 |        /2/t-00017v7/A0002axx.rf_tmp |        [] | {}
   172.20.136.162:9101 (ext) |  31s305ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4;3e00f                                |     1 |        /2/t-0000002/A0002axk.rf_tmp |        [] | {}
   172.20.136.52:9101 (ext) |  30s742ms |  FULL |   USER | 3.35M | 3.35M |       testTable | 2;82013;8                                |     1 |        /2/t-00017iu/A0002axm.rf_tmp |        [] | {}
   ```


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634730815



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -313,6 +291,61 @@ public void run() {
     LOG.info("Shutting down");
   }
 
+  private void updateSummaries() {
+    ExecutorService executor =
+        ThreadPools.createFixedThreadPool(10, "Compaction Summary Gatherer", false);

Review comment:
       I Was thinking of % up to some max (was thinking a max of 10)... and then just decided on the max... that should get through even a large number of tsevers in short order, except when there are a good bit of unresponsive tservers... was also wondering if it should be configurable, but was thinking 10 should be good and we can make it configurable if ever needed




-- 
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.

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



[GitHub] [accumulo] ctubbsii commented on pull request #2096: External Compactions

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


   @dlmarion wrote:
   > I don't think that one of our goals was to create a pluggable external implementation. It's possible it was, I just don't remember that. Is that something we want?
   
   My understanding of the original idea to create an "external" compactions, was to create them "external" to Accumulo... something Accumulo could "submit" work to do and not care about the implementation, and get back a new compacted file. The current implementation only makes them "external" to the tserver, not to Accumulo, and does so by growing Accumulo by 30,000 lines of code. This isn't necessarily bad, but it seems much more complex and tightly coupled to Accumulo's core than I think is maintainable long-term. My understanding of the original idea was to be able to submit compactions to some external service that the user could designate.... it could be a separate process, a bare metal server farm, EC2 nodes that scale on demand, or something else, depending on user needs. My understanding was that there would also be some flexibility in externalizing this functionality, and that implies making it pluggable in some way. Perhaps my understanding was incorrect, but that's what 
 I had thought.
   
   The bare minimum implementation that I was expecting to review at this point would be a pluggable service API for external compactions, and possibly a rudimentary in-process implementation. I wasn't quite expecting new worker processes, and a new coordination service, as well as new ZooKeeper nodes and RPCs. All of that seems to add a lot of complexity, and while that complexity seems necessary for a robust implementation as seems to be provided here (which is great, overall), I worry about maintainability in the future if it isn't modularized and pluggable, or flexibility based on divergent use cases.
   
   Don't get me wrong... I have no issue with the basic design of this PR, as far as I can tell. I'm just wondering how we can improve upon the API/configuration to make it more modular so it's more maintainable in the future, and more flexible for users if they need to swap in alternate implementations of externalized workers to do the actual compacting. Right now, I don't know enough about the implementation to propose suggestions for how to do that... I'm only raising the issue to those who might know. As I try to dive in and learn the implementation, I may have more concrete suggestions. But, this is a lot of code, so it may take time to get that point.


-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   FYI I plan on taking a look at these changes some time this week. Instead of just looking at ALL the changes in the diff, my strategy is to checkout the branch, run some stuff using Uno and then take a look at some of the important files.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633923100



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -0,0 +1,889 @@
+/*
+ * 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.compactor;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator;
+import org.apache.accumulo.core.compaction.thrift.Compactor.Iface;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
+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.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.trace.TraceUtil;
+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.ServerServices;
+import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.core.util.threads.Threads;
+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.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.ServerOpts;
+import org.apache.accumulo.server.compaction.CompactionInfo;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.hadoop.fs.Path;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class Compactor extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.Compactor.Iface {
+
+  public static class CompactorServerOpts extends ServerOpts {
+    @Parameter(required = true, names = {"-q", "--queue"}, description = "compaction queue name")
+    private String queueName = null;
+
+    public String getQueueName() {
+      return queueName;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(Compactor.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long TIME_BETWEEN_CANCEL_CHECKS = 5 * 60 * 1000;
+
+  private static final long TEN_MEGABYTES = 10485760;
+  private static final CompactionCoordinator.Client.Factory COORDINATOR_CLIENT_FACTORY =
+      new CompactionCoordinator.Client.Factory();
+
+  protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  private final UUID compactorId = UUID.randomUUID();
+  private final AccumuloConfiguration aconf;
+  private final String queueName;
+  private final AtomicReference<CompactionCoordinator.Client> coordinatorClient =
+      new AtomicReference<>();
+  protected final AtomicReference<ExternalCompactionId> currentCompactionId =
+      new AtomicReference<>();
+
+  private SecurityOperation security;
+  private ServiceLock compactorLock;
+  private ServerAddress compactorAddress = null;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  protected Compactor(CompactorServerOpts opts, String[] args) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = getConfiguration();
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected Compactor(CompactorServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = conf;
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor,
+      long timeBetweenChecks) {
+    schedExecutor.scheduleWithFixedDelay(() -> checkIfCanceled(), 0, timeBetweenChecks,
+        TimeUnit.MILLISECONDS);
+  }
+
+  protected void checkIfCanceled() {
+    TExternalCompactionJob job = JOB_HOLDER.getJob();
+    if (job != null) {
+      try {
+        var extent = KeyExtent.fromThrift(job.getExtent());
+        var ecid = ExternalCompactionId.of(job.getExternalCompactionId());
+
+        TabletMetadata tabletMeta =
+            getContext().getAmple().readTablet(extent, ColumnType.ECOMP, ColumnType.PREV_ROW);
+        if (tabletMeta == null || !tabletMeta.getExtent().equals(extent)
+            || !tabletMeta.getExternalCompactions().containsKey(ecid)) {
+          // table was deleted OR tablet was split or merged OR tablet no longer thinks compaction
+          // is running for some reason
+          LOG.info("Cancelling compaction {} that no longer has a metadata entry at {}", ecid,

Review comment:
       @ctubbsii I Am currently working through a complete review of the code.  You had mentioned having a very generic per tserver plugin to generally handle external compactions.  Thinking about that when looking at this code, it cancels a running external compaction for a tablet that was deleted (split/merge/table deletion).  To handle this case in a highly pluggable environment, this functionality would need to be present in some form in the SPI.   This would be unrelated to any pluggable component on a tserver, its more related to running external compactions.  The tablet for a running external compaction could be unloaded and then the tablet deleted, no tserver would be aware of 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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   @milleruntime @ctubbsii - I'm looking for an existing IT that tests encryption so I can write something for the external compactions. I have found CryptoTest, which is a unit test, and WriteAheadLogEncryptedIT which doesn't have anything that might be relevant for me. Do you know of an existing test that I can look to for an 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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-849952173


   On a test table in Uno (so a single tserver) w/ 2B entries and 128 tablets I was able to run 40 concurrent external compactions in AKS Kubernetes today (had lots of help from @dlmarion  w/ Kubernetes).  I am continuing to poke at this PR, but I think its looking pretty good.  I hope to run test on cluster+Kubernetes soon instead of Uno+Kubernetes.
   
   ```
   root@uno> listcompactions
    SERVER               | AGE       | TYPE  | REASON | READ  | WROTE | TABLE           | TABLET                                   | INPUT | OUTPUT                              | ITERATORS | ITERATOR OPTIONS
   172.20.128.185:9101 (ext) |   26s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9401a;92012                            |     1 |        /2/t-0000cj0/A0002ay3.rf_tmp |        [] | {}
   172.20.128.188:9101 (ext) |  23s856ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;ba01;b801                              |     1 |        /2/t-00017pv/A0002ayb.rf_tmp |        [] | {}
   172.20.128.41:9101 (ext) |  34s598ms |  FULL |   USER | 3.64M | 3.64M |       testTable | 2;0801b;0602                             |     1 |        /2/t-00004hf/A0002axd.rf_tmp |        [] | {}
   172.20.128.53:9101 (ext) |  35s340ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;02014<                                 |     1 |        /2/t-00017dz/A0002axb.rf_tmp |        [] | {}
   172.20.129.116:9101 (ext) |  23s812ms |  FULL |   USER | 2.40M | 2.40M |       testTable | 2;b801;b601                              |     1 |        /2/t-00004ij/A0002ayc.rf_tmp |        [] | {}
   172.20.129.13:9101 (ext) |   22s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b400b;b200b                            |     1 |        /2/t-0000cjc/A0002aye.rf_tmp |        [] | {}
   172.20.129.184:9101 (ext) |  25s695ms |  FULL |   USER | 2.79M | 2.79M |       testTable | 2;9801c;9602                             |     1 |        /2/t-00004gy/A0002ay7.rf_tmp |        [] | {}
   172.20.129.195:9101 (ext) |  34s670ms |  FULL |   USER | 3.30M | 3.30M |       testTable | 2;0401a;02014                            |     1 |        /2/t-0000cl4/A0002axc.rf_tmp |        [] | {}
   172.20.129.80:9101 (ext) |  28s262ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8c015;8a01e                            |     1 |        /2/t-0000cq8/A0002axv.rf_tmp |        [] | {}
   172.20.130.1:9101 (ext) |  24s517ms |  FULL |   USER | 2.51M | 2.51M |       testTable | 2;be00c;bc00e                            |     1 |        /2/t-00017xu/A0002ay9.rf_tmp |        [] | {}
   172.20.130.104:9101 (ext) |  23s609ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b601;b400b                             |     1 |        /2/t-00017s0/A0002ayd.rf_tmp |        [] | {}
   172.20.130.113:9101 (ext) |  30s713ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;4201;4                                 |     1 |        /2/t-00017kz/A0002axn.rf_tmp |        [] | {}
   172.20.130.201:9101 (ext) |  26s766ms |  FULL |   USER | 2.91M | 2.91M |       testTable | 2;9;8e00d                                |     1 |        /2/t-000000d/A0002ay0.rf_tmp |        [] | {}
   172.20.130.249:9101 (ext) |  28s350ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;46017;44014                            |     1 |        /2/t-00017xf/A0002axu.rf_tmp |        [] | {}
   172.20.131.143:9101 (ext) |  24s460ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;bc00e;ba01                             |     1 |        /2/t-0000cnb/A0002aya.rf_tmp |        [] | {}
   172.20.131.150:9101 (ext) |  25s101ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9a01f;9801c                            |     1 |        /2/t-00017qv/A0002ay8.rf_tmp |        [] | {}
   172.20.131.35:9101 (ext) |  26s969ms |  FULL |   USER | 2.77M | 2.77M |       testTable | 2;48016;46017                            |     1 |        /2/t-00004ho/A0002axz.rf_tmp |        [] | {}
   172.20.131.82:9101 (ext) |  33s715ms |  FULL |   USER | 3.36M | 3.36M |       testTable | 2;0c01a;0a01d                            |     1 |        /2/t-0000ctq/A0002axg.rf_tmp |        [] | {}
   172.20.132.14:9101 (ext) |  25s729ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4c01;4a018                             |     1 |        /2/t-0000col/A0002ay6.rf_tmp |        [] | {}
   172.20.132.141:9101 (ext) |  28s416ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8a01e;8801d                            |     1 |        /2/t-00017so/A0002axt.rf_tmp |        [] | {}
   172.20.132.187:9101 (ext) |  28s737ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8801d;86021                            |     1 |        /2/t-00004gp/A0002axs.rf_tmp |        [] | {}
   172.20.132.226:9101 (ext) |  30s934ms |  FULL |   USER | 4.30M | 4.30M |       testTable | 2;8;7e00d                                |     1 |        /2/t-0000000/A0002axl.rf_tmp |        [] | {}
   172.20.132.26:9101 (ext) |  29s994ms |  FULL |   USER | 3.17M | 3.17M |       testTable | 2;22016;2                                |     1 |        /2/t-00017lc/A0002axq.rf_tmp |        [] | {}
   172.20.133.113:9101 (ext) |  29s711ms |  FULL |   USER | 3.20M | 3.20M |       testTable | 2;44014;4201                             |     1 |        /2/t-0000clm/A0002axr.rf_tmp |        [] | {}
   172.20.133.144:9101 (ext) |  32s707ms |  FULL |   USER | 4.10M | 4.10M |       testTable | 2;1;0e013                                |     1 |        /2/t-0000008/A0002axi.rf_tmp |        [] | {}
   172.20.133.182:9101 (ext) |  28s313ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8e00d;8c015                            |     1 |        /2/t-00017zq/A0002axw.rf_tmp |        [] | {}
   172.20.133.255:9101 (ext) |  26s703ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;92012;9                                |     1 |        /2/t-00017kk/A0002ay1.rf_tmp |        [] | {}
   172.20.133.55:9101 (ext) |  25s724ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;26024;2401d                            |     1 |        /2/t-00017us/A0002ay5.rf_tmp |        [] | {}
   172.20.134.114:9101 (ext) |  26s191ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4a018;48016                            |     1 |        /2/t-00017wj/A0002ay2.rf_tmp |        [] | {}
   172.20.134.139:9101 (ext) |  25s993ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9602;9401a                             |     1 |        /2/t-00017rl/A0002ay4.rf_tmp |        [] | {}
   172.20.134.161:9101 (ext) |  30s202ms |  FULL |   USER | 3.70M | 3.70M |       testTable | 2;86021;84019                            |     1 |        /2/t-00017hh/A0002axp.rf_tmp |        [] | {}
   172.20.134.250:9101 (ext) |  30s186ms |  FULL |   USER | 3.19M | 3.19M |       testTable | 2;84019;82013                            |     1 |        /2/t-0000co6/A0002axo.rf_tmp |        [] | {}
   172.20.134.61:9101 (ext) |   34s46ms |  FULL |   USER | 4.52M | 4.52M |       testTable | 2;0602;0401a                             |     1 |        /2/t-00017mi/A0002axf.rf_tmp |        [] | {}
   172.20.135.166:9101 (ext) |   33s46ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;0e013;0c01a                            |     1 |        /2/t-0001816/A0002axh.rf_tmp |        [] | {}
   172.20.135.191:9101 (ext) |  32s423ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2;1e00e                                |     1 |        /2/t-0000005/A0002axj.rf_tmp |        [] | {}
   172.20.135.222:9101 (ext) |  34s165ms |  FULL |   USER | 4.43M | 4.43M |       testTable | 2;0a01d;0801b                            |     1 |        /2/t-00017gs/A0002axe.rf_tmp |        [] | {}
   172.20.135.41:9101 (ext) |  26s997ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2401d;22016                            |     1 |        /2/t-0000cle/A0002axy.rf_tmp |        [] | {}
   172.20.136.109:9101 (ext) |  27s177ms |  FULL |   USER | 2.98M | 2.98M |       testTable | 2;1201;1                                 |     1 |        /2/t-00017v7/A0002axx.rf_tmp |        [] | {}
   172.20.136.162:9101 (ext) |  31s305ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4;3e00f                                |     1 |        /2/t-0000002/A0002axk.rf_tmp |        [] | {}
   172.20.136.52:9101 (ext) |  30s742ms |  FULL |   USER | 3.35M | 3.35M |       testTable | 2;82013;8                                |     1 |        /2/t-00017iu/A0002axm.rf_tmp |        [] | {}
   ```


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633775560



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =

Review comment:
       I wonder if the DeadCompactionDetector should have a reference to the CompactionCoordinator and call `compactionFailed` instead of calling the CompactionFinalizer directly, that way the behavior and logic are the same as a normal failure.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632084089



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.compaction;
+
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {

Review comment:
       To give some context using the DefaultPlanner as an example.  It will make SPI calls that give it a CompactionExecutorId like [here](https://github.com/apache/accumulo/blob/129d06b66ca52307a77f12cee203b70e3604585a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java#L126).  In never instantiates a CompactionExecutorId.  Then later it will pass those IDs back when planning like [here](https://github.com/apache/accumulo/blob/129d06b66ca52307a77f12cee203b70e3604585a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java#L240-L242)




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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






-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   > @milleruntime - what's the best way to approach that? Create a branch off of this branch, remove the thrift files from this branch and remove the non-thrift files from the other?
   
   I was thinking remove the thrift files, do a typical build, commit your changes and then build again with the `-Pthrift` and then commit the generated files. Hopefully you still have the thrift executable!


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633450856



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       Resolved in 011a861832036fd22d0a8ee55e4daa2e11c90801




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632109671



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
##########
@@ -204,6 +210,54 @@ public boolean testClassLoad(final String className, final String asTypeName)
     }
   }
 
+  @Override
+  public List<ActiveCompaction> getActiveCompactions()
+      throws AccumuloException, AccumuloSecurityException {
+
+    List<HostAndPort> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    List<String> tservers = getTabletServers();
+
+    int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
+    var executorService =
+        ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions", false);
+    try {
+      List<Future<List<ActiveCompaction>>> futures = new ArrayList<>();
+
+      for (String tserver : tservers) {
+        futures.add(executorService.submit(() -> getActiveCompactions(tserver)));
+      }
+
+      for (HostAndPort compactorAddr : compactors) {
+        futures
+            .add(
+                executorService
+                    .submit(
+                        () -> ExternalCompactionUtil.getActiveCompaction(compactorAddr, context)
+                            .stream().map(tac -> new ActiveCompactionImpl(context, tac,
+                                compactorAddr, CompactionHost.Type.COMPACTOR))
+                            .collect(Collectors.toList())));

Review comment:
       Improved this a bit in 162dd806b1754f890c4f139a3611c54024c072a2.  Tried to rename the var `tac` to `thriftActiveCompaction` but it caused the formatting of the update code to be unsightly.




-- 
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.

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



[GitHub] [accumulo] Manno15 commented on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844945493


   > @Manno15 I think I may have fixed the issue you were seeing in 3095d3d, if line 1055 was the !closed check. That was an incorrect sanity check, looking at how the close() method works, closed could be true there and that would be ok.
   
   Yeah, that is the line I saw for 1055. I was running the latest commit on this branch at that time.
   
   > @Manno15 the feedback on logging was great. I made some quick changes with the following goals.
   
   Great, I like the logging changes you made. I will test them out today. 
   
   > @Manno15 re the consistency of those fields, that data was coming from a thrift class and GSON generator json, so not sure they could reliably be ordered consistently.
   
   Ah, that makes sense. It's not a huge deal, just something I noticed while reading through it.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632701484



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }
+
+          // As a fallback, try them all
+          if (!matchFound) {
+            LOG.debug("Checking all tservers for external running compaction, extent: {}", extent);
+            for (TServerInstance tsi : tservers) {
+              TabletClientService.Client client = null;
+              try {
+                client = getTabletServerConnection(tsi);
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                }
+              } catch (TException e) {
+                LOG.error(
+                    "Error from tserver {} while trying to check if external compaction is running, trying next tserver",
+                    ExternalCompactionUtil.getHostPortString(tsi.getHostAndPort()), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            }
+          }
+
+          if (!matchFound) {
+            LOG.warn(
+                "There is an external compaction running on a compactor, but could not find corresponding tablet server. Extent: {}, Compactor: {}, Compaction: {}",
+                extent, hp, job);
+          }
+        });
+      }
+      tservers.clear();
+    } else {
+      LOG.info("No running tablet servers found, continuing startup");
+    }
+
+    tserverSet.startListeningForTabletServerChanges();
+    startDeadCompactionDetector();
+
+    LOG.info("Starting loop to check tservers for compaction summaries");
+    while (!shutdown) {
+      long start = System.currentTimeMillis();
+      tserverSet.getCurrentServers().forEach(tsi -> {
+        try {
+          TabletClientService.Client client = null;
+          try {
+            LOG.debug("Contacting tablet server {} to get external compaction summaries",
+                tsi.getHostPort());
+            client = getTabletServerConnection(tsi);
+            List<TCompactionQueueSummary> summaries =
+                client.getCompactionQueueInfo(TraceUtil.traceInfo(), getContext().rpcCreds());
+            summaries.forEach(summary -> {
+              QueueAndPriority qp =
+                  QueueAndPriority.get(summary.getQueue().intern(), summary.getPriority());
+              synchronized (qp) {
+                TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(qp.getQueue(), k -> 0L);
+                QUEUE_SUMMARIES.update(tsi, summaries);
+              }
+            });
+          } finally {
+            ThriftUtil.returnClient(client);
+          }
+        } catch (TException e) {
+          LOG.warn("Error getting external compaction summaries from tablet server: {}",
+              tsi.getHostAndPort(), e);
+          QUEUE_SUMMARIES.remove(Set.of(tsi));
+        }
+      });
+
+      long now = System.currentTimeMillis();
+      TIME_COMPACTOR_LAST_CHECKED.forEach((k, v) -> {
+        if ((now - v) > getMissingCompactorWarningTime()) {
+          LOG.warn("No compactors have checked in with coordinator for queue {} in {}ms", k,
+              getMissingCompactorWarningTime());
+        }
+      });
+
+      long checkInterval = getTServerCheckInterval();
+      long duration = (System.currentTimeMillis() - start);
+      if (checkInterval - duration > 0) {
+        LOG.debug("Waiting {}ms for next tserver check", (checkInterval - duration));
+        UtilWaitThread.sleep(checkInterval - duration);
+      }
+    }
+
+    LOG.info("Shutting down");
+  }
+
+  protected void startDeadCompactionDetector() {
+    new DeadCompactionDetector(getContext(), compactionFinalizer, schedExecutor).start();
+  }
+
+  protected long getMissingCompactorWarningTime() {
+    return FIFTEEN_MINUTES;
+  }
+
+  protected long getTServerCheckInterval() {
+    return this.aconf.getTimeInMillis(Property.COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL);
+  }
+
+  protected TabletMetadata getMetadataEntryForExtent(KeyExtent extent) {
+    return getContext().getAmple().readTablets().forTablet(extent)
+        .fetch(ColumnType.LOCATION, ColumnType.PREV_ROW).build().stream().findFirst().orElse(null);
+  }
+
+  /**
+   * Callback for the LiveTServerSet object to update current set of tablet servers, including ones
+   * that were deleted and added
+   *
+   * @param current
+   *          current set of live tservers
+   * @param deleted
+   *          set of tservers that were removed from current since last update
+   * @param added
+   *          set of tservers that were added to current since last update
+   */
+  @Override
+  public void update(LiveTServerSet current, Set<TServerInstance> deleted,
+      Set<TServerInstance> added) {
+
+    // run() will iterate over the current and added tservers and add them to the internal
+    // data structures. For tservers that are deleted, we need to remove them from QUEUES
+    // and INDEX
+    QUEUE_SUMMARIES.remove(deleted);
+  }
+
+  /**
+   * Return the next compaction job from the queue to a Compactor
+   *
+   * @param queueName
+   *          queue
+   * @param compactorAddress
+   *          compactor address
+   * @return compaction job
+   */
+  @Override
+  public TExternalCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials,
+      String queueName, String compactorAddress, String externalCompactionId) throws TException {
+
+    // do not expect users to call this directly, expect compactors to call this method
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+    final String queue = queueName.intern();
+    LOG.debug("getCompactionJob called for queue {} by compactor {}", queue, compactorAddress);
+    TIME_COMPACTOR_LAST_CHECKED.put(queue, System.currentTimeMillis());
+
+    TExternalCompactionJob result = null;
+
+    PrioTserver prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+
+    while (prioTserver != null) {
+      TServerInstance tserver = prioTserver.tserver;
+
+      LOG.debug("Getting compaction for queue {} from tserver {}", queue, tserver.getHostAndPort());
+      // Get a compaction from the tserver
+      TabletClientService.Client client = null;
+      try {
+        client = getTabletServerConnection(tserver);
+        TExternalCompactionJob job =
+            client.reserveCompactionJob(TraceUtil.traceInfo(), getContext().rpcCreds(), queue,
+                prioTserver.prio, compactorAddress, externalCompactionId);
+        if (null == job.getExternalCompactionId()) {
+          LOG.debug("No compactions found for queue {} on tserver {}, trying next tserver", queue,
+              tserver.getHostAndPort(), compactorAddress);
+
+          QUEUE_SUMMARIES.removeSummary(tserver, queueName, prioTserver.prio);
+          prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+          continue;
+        }
+        RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+            new RunningCompaction(job, compactorAddress, tserver));
+        LOG.debug("Returning external job {} to {}", job.externalCompactionId, compactorAddress);
+        result = job;
+        break;
+      } catch (TException e) {
+        LOG.warn("Error from tserver {} while trying to reserve compaction, trying next tserver",
+            ExternalCompactionUtil.getHostPortString(tserver.getHostAndPort()), e);
+        QUEUE_SUMMARIES.removeSummary(tserver, queueName, prioTserver.prio);
+        prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+      } finally {
+        ThriftUtil.returnClient(client);
+      }
+    }
+
+    if (result == null) {
+      LOG.debug("No tservers found for queue {}, returning empty job to compactor {}", queue,
+          compactorAddress);
+      result = new TExternalCompactionJob();
+    }
+
+    return result;
+
+  }
+
+  /**
+   * Return the Thrift client for the TServer
+   *
+   * @param tserver
+   *          tserver instance
+   * @return thrift client
+   * @throws TTransportException
+   *           thrift error
+   */
+  protected TabletClientService.Client getTabletServerConnection(TServerInstance tserver)
+      throws TTransportException {
+    TServerConnection connection = tserverSet.getConnection(tserver);
+    TTransport transport =
+        ThriftTransportPool.getInstance().getTransport(connection.getAddress(), 0, getContext());
+    return ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+  }
+
+  /**
+   * Return the Thrift client for the Compactor
+   *
+   * @param compactorAddress
+   *          compactor address
+   * @return thrift client
+   * @throws TTransportException
+   *           thrift error
+   */
+  protected Compactor.Client getCompactorConnection(HostAndPort compactorAddress)
+      throws TTransportException {
+    TTransport transport =
+        ThriftTransportPool.getInstance().getTransport(compactorAddress, 0, getContext());
+    return ThriftUtil.createClient(new Compactor.Client.Factory(), transport);
+  }
+
+  /**
+   * Called by the TabletServer to cancel the running compaction.
+   *
+   * @param tinfo
+   *          trace info
+   * @param credentials
+   *          tcredentials object
+   * @param externalCompactionId
+   *          compaction id
+   * @throws TException
+   *           thrift error
+   */
+  @Override
+  public void cancelCompaction(TInfo tinfo, TCredentials credentials, String externalCompactionId)

Review comment:
       It seems like nothing calls this, however its tricky to know for sure w/ thrift.  If not used then maybe it could be removed.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =

Review comment:
       Maybe the dead compaction detector should remove things from here when it finds a dead external compaction.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       Seems like there may be vestigial code here, the code gets the tserver instance and passes it to RunningCompaction() but its never used after that.  Also if the metadata entry has a corresponding external compaction entry then the tserver thinks the compaction is running so may be able to use that instead of contacting tserver, not completely sure though.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.coordinator;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+
+public class RunningCompaction {
+
+  private final TExternalCompactionJob job;
+  private final String compactorAddress;
+  private final TServerInstance tserver;
+  private final Map<Long,CompactionUpdate> updates = new TreeMap<>();
+
+  RunningCompaction(TExternalCompactionJob job, String compactorAddress, TServerInstance tserver) {
+    super();
+    this.job = job;
+    this.compactorAddress = compactorAddress;
+    this.tserver = tserver;
+  }
+
+  public Map<Long,CompactionUpdate> getUpdates() {
+    return updates;
+  }
+
+  public void addUpdate(Long timestamp, String message, TCompactionState state) {
+    this.updates.put(timestamp, new CompactionUpdate(timestamp, message, state));
+  }
+
+  public TExternalCompactionJob getJob() {
+    return job;
+  }
+
+  public String getCompactorAddress() {
+    return compactorAddress;
+  }
+
+  public TServerInstance getTserver() {

Review comment:
       This may not be used, could possibly be removed.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }
+
+          // As a fallback, try them all
+          if (!matchFound) {
+            LOG.debug("Checking all tservers for external running compaction, extent: {}", extent);
+            for (TServerInstance tsi : tservers) {
+              TabletClientService.Client client = null;
+              try {
+                client = getTabletServerConnection(tsi);
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                }
+              } catch (TException e) {
+                LOG.error(
+                    "Error from tserver {} while trying to check if external compaction is running, trying next tserver",
+                    ExternalCompactionUtil.getHostPortString(tsi.getHostAndPort()), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            }
+          }
+
+          if (!matchFound) {
+            LOG.warn(
+                "There is an external compaction running on a compactor, but could not find corresponding tablet server. Extent: {}, Compactor: {}, Compaction: {}",
+                extent, hp, job);
+          }
+        });
+      }
+      tservers.clear();
+    } else {
+      LOG.info("No running tablet servers found, continuing startup");
+    }
+
+    tserverSet.startListeningForTabletServerChanges();
+    startDeadCompactionDetector();
+
+    LOG.info("Starting loop to check tservers for compaction summaries");
+    while (!shutdown) {
+      long start = System.currentTimeMillis();
+      tserverSet.getCurrentServers().forEach(tsi -> {
+        try {
+          TabletClientService.Client client = null;
+          try {
+            LOG.debug("Contacting tablet server {} to get external compaction summaries",
+                tsi.getHostPort());
+            client = getTabletServerConnection(tsi);
+            List<TCompactionQueueSummary> summaries =
+                client.getCompactionQueueInfo(TraceUtil.traceInfo(), getContext().rpcCreds());
+            summaries.forEach(summary -> {
+              QueueAndPriority qp =
+                  QueueAndPriority.get(summary.getQueue().intern(), summary.getPriority());
+              synchronized (qp) {
+                TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(qp.getQueue(), k -> 0L);
+                QUEUE_SUMMARIES.update(tsi, summaries);
+              }
+            });
+          } finally {
+            ThriftUtil.returnClient(client);
+          }

Review comment:
       Wondering if these RPCs should be done in a thread pool so that multiple tserver can be contacted in parallel.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.server.compaction;
+
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RetryableThriftCall<T> {
+
+  public static class RetriesExceededException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    public RetriesExceededException() {
+      super();
+    }
+
+    public RetriesExceededException(String message, Throwable cause, boolean enableSuppression,
+        boolean writableStackTrace) {
+      super(message, cause, enableSuppression, writableStackTrace);
+    }
+
+    public RetriesExceededException(String message, Throwable cause) {
+      super(message, cause);
+    }
+
+    public RetriesExceededException(String message) {
+      super(message);
+    }
+
+    public RetriesExceededException(Throwable cause) {
+      super(cause);
+    }
+
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(RetryableThriftCall.class);
+  public static final long MAX_WAIT_TIME = 60000;
+
+  private final long start;
+  private final long maxWaitTime;
+  private int maxNumRetries;
+  private final RetryableThriftFunction<T> function;
+  private final boolean retryForever;
+
+  /**
+   * RetryableThriftCall constructor
+   *
+   * @param start
+   *          initial wait time
+   * @param maxWaitTime
+   *          max wait time
+   * @param maxNumRetries
+   *          number of times to retry, 0 to retry forever
+   * @param function
+   *          function to execute
+   */
+  public RetryableThriftCall(long start, long maxWaitTime, int maxNumRetries,
+      RetryableThriftFunction<T> function) {
+    this.start = start;
+    this.maxWaitTime = maxWaitTime;
+    this.maxNumRetries = maxNumRetries;
+    this.function = function;
+    this.retryForever = (maxNumRetries == 0);
+  }
+
+  /**
+   * Attempts to call the function, waiting and retrying when TException is thrown. Wait time is
+   * initially set to the start time and doubled each time, up to the maximum wait time. If
+   * maxNumRetries is 0, then this will retry forever. If maxNumRetries is non-zero, then a
+   * RuntimeException is thrown when it has exceeded he maxNumRetries parameter.
+   *
+   * @return T
+   * @throws RetriesExceededException
+   *           when maximum number of retries has been exceeded and the cause is set to the last
+   *           TException
+   */
+  public T run() throws RetriesExceededException {
+    long waitTime = start;

Review comment:
       We could possible use [Retry](https://github.com/apache/accumulo/blob/0c8be59a6c1a8024e3af9c2790ea58fc69c1ae7e/core/src/main/java/org/apache/accumulo/fate/util/Retry.java#L38) in the implementation of this 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.

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



[GitHub] [accumulo] ctubbsii commented on pull request #2096: External Compactions

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


   Based on the documentation in apache/accumulo-website#282 , it seems that the way the external compactions are exposed through to the user in the compaction configuration is through the specially named `externalQueue`, which seems to be a special feature of the default compaction service. Is that correct?
   
   From the perspective of the user configuring things, it seems difficult to reason about how this configuration maps to pluggable services that they are deploying and administering, because the external compaction services aren't configured explicitly, but are an internal feature of a monolithic compaction service implementation. It seems like it will try to "automagically" use external services because of a specially named queue, and not because some plugin like `my.external.compaction.CompactionExecutorService` is configured explicitly as an executor (unless I'm mistaken here; I'm still going through the code).
   
   I'm wondering if there are ways this could be made a bit more modular and the configuration a bit more explicit, so that the default compaction service isn't so monolithic, and users can very explicitly swap out the external executor implementation for another one. This specific implementation could be maintained in a side repository, deployed as a separate optional jar, and configured by users by some class name at an appropriate place in the configuration that specifies a plugin implementation. However, it should also be capable of being easily swapped out for a different external compaction implementation, and I'm not sure that's the case right now.
   
   My main concern is that if the implementation is too monolithic, it creates a maintenance burden to continue to add marginal improvements to the monolithic implementation to satisfy every divergent use case, rather than encourage users to take advantage of the modular nature of our plugins to develop implementations to satisfy specific use cases. Ideally, we'd really want to make things pluggable/modular and the plugin interfaces relatively lightweight.


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   In the current website PR I removed our original design document because @keith-turner and I had made several changes over the course of development. However, I still have the original documentation in another branch. I revisited our [original](https://github.com/dlmarion/accumulo-website/blob/external-compaction-design-capture-all-information/design/external-compaction.md) design and I think what we have today is closely aligned with what we envisioned at the beginning of this process. Having the benefit of already writing the external compaction code, I'm not sure that a separate and independent service  could be achieved without a significant set of new APIs being written for Accumulo.
   
   I think @keith-turner pointed to the planner being the place where someone can write their own compaction implementation. But, I do agree that this initial implementation is tightly coupled to Accumulo internals. Maybe this is a stepping stone to a fully pluggable implementation. Looking at the coordinator should tell us what it needs in a public API to make it fully external.
   
   Finally, my personal goal here was to move compactions out of the TServer for several reasons (below). The Compactor component can be run on different hardware than the TabletServers, and even run in Kubernetes using their dynamic pod scheduling feature to scale up/down the number of Compactors based on load.
    
     * Allow compactions to outlive a TabletServer
     * Allow compactions to occur concurrent to a Tablet being re-hosted
     * Reduce the load on the TabletServer, giving it more cycles to insert mutations and respond to scans
     * Allow compactions to be scaled differently than the number of 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.

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



[GitHub] [accumulo] Manno15 commented on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844253910


   One other very minor thing (and I do mean very minor, feel free to ignore this). I noticed with the `CompactionCoordinator` and `CompactionFinalizer` both have filesize and entries parameters but they are in the opposite order as each other. When reading the logs, it would be more consistent if they matched (since they come one after another in the logs). The screenshot below shows the log messages I am referring to.
   
   ![image](https://user-images.githubusercontent.com/29436247/118846534-ce447780-b89a-11eb-81c3-8075ff14c20d.png)
   


-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-849952173


   On a test table in Uno (so a single tserver) w/ 2B entries and 128 tablets I was able to run 40 concurrent external compactions in AKS Kubernetes today (had lots of help from @dlmarion  w/ Kubernetes).  I am continuing to poke at this PR, but I think its looking pretty good.  I hope to run test on cluster+Kubernetes soon instead of Uno(single VM)+Kubernetes.
   
   ```
   root@uno> listcompactions
    SERVER               | AGE       | TYPE  | REASON | READ  | WROTE | TABLE           | TABLET                                   | INPUT | OUTPUT                              | ITERATORS | ITERATOR OPTIONS
   172.20.128.185:9101 (ext) |   26s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9401a;92012                            |     1 |        /2/t-0000cj0/A0002ay3.rf_tmp |        [] | {}
   172.20.128.188:9101 (ext) |  23s856ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;ba01;b801                              |     1 |        /2/t-00017pv/A0002ayb.rf_tmp |        [] | {}
   172.20.128.41:9101 (ext) |  34s598ms |  FULL |   USER | 3.64M | 3.64M |       testTable | 2;0801b;0602                             |     1 |        /2/t-00004hf/A0002axd.rf_tmp |        [] | {}
   172.20.128.53:9101 (ext) |  35s340ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;02014<                                 |     1 |        /2/t-00017dz/A0002axb.rf_tmp |        [] | {}
   172.20.129.116:9101 (ext) |  23s812ms |  FULL |   USER | 2.40M | 2.40M |       testTable | 2;b801;b601                              |     1 |        /2/t-00004ij/A0002ayc.rf_tmp |        [] | {}
   172.20.129.13:9101 (ext) |   22s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b400b;b200b                            |     1 |        /2/t-0000cjc/A0002aye.rf_tmp |        [] | {}
   172.20.129.184:9101 (ext) |  25s695ms |  FULL |   USER | 2.79M | 2.79M |       testTable | 2;9801c;9602                             |     1 |        /2/t-00004gy/A0002ay7.rf_tmp |        [] | {}
   172.20.129.195:9101 (ext) |  34s670ms |  FULL |   USER | 3.30M | 3.30M |       testTable | 2;0401a;02014                            |     1 |        /2/t-0000cl4/A0002axc.rf_tmp |        [] | {}
   172.20.129.80:9101 (ext) |  28s262ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8c015;8a01e                            |     1 |        /2/t-0000cq8/A0002axv.rf_tmp |        [] | {}
   172.20.130.1:9101 (ext) |  24s517ms |  FULL |   USER | 2.51M | 2.51M |       testTable | 2;be00c;bc00e                            |     1 |        /2/t-00017xu/A0002ay9.rf_tmp |        [] | {}
   172.20.130.104:9101 (ext) |  23s609ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b601;b400b                             |     1 |        /2/t-00017s0/A0002ayd.rf_tmp |        [] | {}
   172.20.130.113:9101 (ext) |  30s713ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;4201;4                                 |     1 |        /2/t-00017kz/A0002axn.rf_tmp |        [] | {}
   172.20.130.201:9101 (ext) |  26s766ms |  FULL |   USER | 2.91M | 2.91M |       testTable | 2;9;8e00d                                |     1 |        /2/t-000000d/A0002ay0.rf_tmp |        [] | {}
   172.20.130.249:9101 (ext) |  28s350ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;46017;44014                            |     1 |        /2/t-00017xf/A0002axu.rf_tmp |        [] | {}
   172.20.131.143:9101 (ext) |  24s460ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;bc00e;ba01                             |     1 |        /2/t-0000cnb/A0002aya.rf_tmp |        [] | {}
   172.20.131.150:9101 (ext) |  25s101ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9a01f;9801c                            |     1 |        /2/t-00017qv/A0002ay8.rf_tmp |        [] | {}
   172.20.131.35:9101 (ext) |  26s969ms |  FULL |   USER | 2.77M | 2.77M |       testTable | 2;48016;46017                            |     1 |        /2/t-00004ho/A0002axz.rf_tmp |        [] | {}
   172.20.131.82:9101 (ext) |  33s715ms |  FULL |   USER | 3.36M | 3.36M |       testTable | 2;0c01a;0a01d                            |     1 |        /2/t-0000ctq/A0002axg.rf_tmp |        [] | {}
   172.20.132.14:9101 (ext) |  25s729ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4c01;4a018                             |     1 |        /2/t-0000col/A0002ay6.rf_tmp |        [] | {}
   172.20.132.141:9101 (ext) |  28s416ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8a01e;8801d                            |     1 |        /2/t-00017so/A0002axt.rf_tmp |        [] | {}
   172.20.132.187:9101 (ext) |  28s737ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8801d;86021                            |     1 |        /2/t-00004gp/A0002axs.rf_tmp |        [] | {}
   172.20.132.226:9101 (ext) |  30s934ms |  FULL |   USER | 4.30M | 4.30M |       testTable | 2;8;7e00d                                |     1 |        /2/t-0000000/A0002axl.rf_tmp |        [] | {}
   172.20.132.26:9101 (ext) |  29s994ms |  FULL |   USER | 3.17M | 3.17M |       testTable | 2;22016;2                                |     1 |        /2/t-00017lc/A0002axq.rf_tmp |        [] | {}
   172.20.133.113:9101 (ext) |  29s711ms |  FULL |   USER | 3.20M | 3.20M |       testTable | 2;44014;4201                             |     1 |        /2/t-0000clm/A0002axr.rf_tmp |        [] | {}
   172.20.133.144:9101 (ext) |  32s707ms |  FULL |   USER | 4.10M | 4.10M |       testTable | 2;1;0e013                                |     1 |        /2/t-0000008/A0002axi.rf_tmp |        [] | {}
   172.20.133.182:9101 (ext) |  28s313ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8e00d;8c015                            |     1 |        /2/t-00017zq/A0002axw.rf_tmp |        [] | {}
   172.20.133.255:9101 (ext) |  26s703ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;92012;9                                |     1 |        /2/t-00017kk/A0002ay1.rf_tmp |        [] | {}
   172.20.133.55:9101 (ext) |  25s724ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;26024;2401d                            |     1 |        /2/t-00017us/A0002ay5.rf_tmp |        [] | {}
   172.20.134.114:9101 (ext) |  26s191ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4a018;48016                            |     1 |        /2/t-00017wj/A0002ay2.rf_tmp |        [] | {}
   172.20.134.139:9101 (ext) |  25s993ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9602;9401a                             |     1 |        /2/t-00017rl/A0002ay4.rf_tmp |        [] | {}
   172.20.134.161:9101 (ext) |  30s202ms |  FULL |   USER | 3.70M | 3.70M |       testTable | 2;86021;84019                            |     1 |        /2/t-00017hh/A0002axp.rf_tmp |        [] | {}
   172.20.134.250:9101 (ext) |  30s186ms |  FULL |   USER | 3.19M | 3.19M |       testTable | 2;84019;82013                            |     1 |        /2/t-0000co6/A0002axo.rf_tmp |        [] | {}
   172.20.134.61:9101 (ext) |   34s46ms |  FULL |   USER | 4.52M | 4.52M |       testTable | 2;0602;0401a                             |     1 |        /2/t-00017mi/A0002axf.rf_tmp |        [] | {}
   172.20.135.166:9101 (ext) |   33s46ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;0e013;0c01a                            |     1 |        /2/t-0001816/A0002axh.rf_tmp |        [] | {}
   172.20.135.191:9101 (ext) |  32s423ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2;1e00e                                |     1 |        /2/t-0000005/A0002axj.rf_tmp |        [] | {}
   172.20.135.222:9101 (ext) |  34s165ms |  FULL |   USER | 4.43M | 4.43M |       testTable | 2;0a01d;0801b                            |     1 |        /2/t-00017gs/A0002axe.rf_tmp |        [] | {}
   172.20.135.41:9101 (ext) |  26s997ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2401d;22016                            |     1 |        /2/t-0000cle/A0002axy.rf_tmp |        [] | {}
   172.20.136.109:9101 (ext) |  27s177ms |  FULL |   USER | 2.98M | 2.98M |       testTable | 2;1201;1                                 |     1 |        /2/t-00017v7/A0002axx.rf_tmp |        [] | {}
   172.20.136.162:9101 (ext) |  31s305ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4;3e00f                                |     1 |        /2/t-0000002/A0002axk.rf_tmp |        [] | {}
   172.20.136.52:9101 (ext) |  30s742ms |  FULL |   USER | 3.35M | 3.35M |       testTable | 2;82013;8                                |     1 |        /2/t-00017iu/A0002axm.rf_tmp |        [] | {}
   ```


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632084089



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.compaction;
+
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {

Review comment:
       To give some context using the DefaultPlanner as an example.  It will make SPI calls that give it a CompactionExecutorId like [here](https://github.com/apache/accumulo/blob/129d06b66ca52307a77f12cee203b70e3604585a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java#L126).  It never instantiates a CompactionExecutorId.  Then later it will pass those IDs back when planning like [here](https://github.com/apache/accumulo/blob/129d06b66ca52307a77f12cee203b70e3604585a/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java#L240-L242)




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > Everything works as expected from my testing. Ran numerous tests with varying parameters (with and without agitation, multi-node cluster, and varying amounts of ingested data).
   
   Nice, did you verify the data after testing?


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633491566



##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.server.compaction;
+
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RetryableThriftCall<T> {
+
+  public static class RetriesExceededException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    public RetriesExceededException() {
+      super();
+    }
+
+    public RetriesExceededException(String message, Throwable cause, boolean enableSuppression,
+        boolean writableStackTrace) {
+      super(message, cause, enableSuppression, writableStackTrace);
+    }
+
+    public RetriesExceededException(String message, Throwable cause) {
+      super(message, cause);
+    }
+
+    public RetriesExceededException(String message) {
+      super(message);
+    }
+
+    public RetriesExceededException(Throwable cause) {
+      super(cause);
+    }
+
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(RetryableThriftCall.class);
+  public static final long MAX_WAIT_TIME = 60000;
+
+  private final long start;
+  private final long maxWaitTime;
+  private int maxNumRetries;
+  private final RetryableThriftFunction<T> function;
+  private final boolean retryForever;
+
+  /**
+   * RetryableThriftCall constructor
+   *
+   * @param start
+   *          initial wait time
+   * @param maxWaitTime
+   *          max wait time
+   * @param maxNumRetries
+   *          number of times to retry, 0 to retry forever
+   * @param function
+   *          function to execute
+   */
+  public RetryableThriftCall(long start, long maxWaitTime, int maxNumRetries,
+      RetryableThriftFunction<T> function) {
+    this.start = start;
+    this.maxWaitTime = maxWaitTime;
+    this.maxNumRetries = maxNumRetries;
+    this.function = function;
+    this.retryForever = (maxNumRetries == 0);
+  }
+
+  /**
+   * Attempts to call the function, waiting and retrying when TException is thrown. Wait time is
+   * initially set to the start time and doubled each time, up to the maximum wait time. If
+   * maxNumRetries is 0, then this will retry forever. If maxNumRetries is non-zero, then a
+   * RuntimeException is thrown when it has exceeded he maxNumRetries parameter.
+   *
+   * @return T
+   * @throws RetriesExceededException
+   *           when maximum number of retries has been exceeded and the cause is set to the last
+   *           TException
+   */
+  public T run() throws RetriesExceededException {
+    long waitTime = start;

Review comment:
       Resolved in 3983cd23629789c1ed0020d01fdd968eabfe590f




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635585325



##########
File path: core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java
##########
@@ -31,4 +31,9 @@
    * Create a thread pool executor within a compaction service.
    */
   public CompactionExecutorId createExecutor(String name, int threads);
+
+  /**
+   * @return an id for a configured external execution queue.
+   */
+  public CompactionExecutorId getExternalExecutor(String name);

Review comment:
       The interface was not there so that users could implement their own, I was just using it to logically group things that went together to avoid having lots of unrelated groups of methods interleaved in a single interface.  Also createExecutor did not feel like a parameter 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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r636159684



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(
+      Map<ExternalCompactionId,ExternalCompactionMetadata> extCompactions, Tablet tablet,
+      Map<ExternalCompactionId,String> externalCompactionsToRemove) {
+    CompactionKind extKind = null;
+    boolean unexpectedExternal = false;
+    Set<StoredTabletFile> tmpSelectedFiles = null;
+    Boolean selAll = null;
+    Long cid = null;
+    Boolean propDel = null;
+    int count = 0;
+
+    ArrayList<String> reasons = new ArrayList<>();
+
+    for (Entry<ExternalCompactionId,ExternalCompactionMetadata> entry : extCompactions.entrySet()) {

Review comment:
       Sure @milleruntime that would be helpful.  Was thinking those test could be done after this is merged but before release.  It would be easier to review 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.

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



[GitHub] [accumulo] Manno15 edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-845899886


   I also noticed that this log message below will repeat indefinitely in the right scenario (one Tserver cluster that has the Tserver crash or killed). In that scenario, it will repeat this log message until one TServer is brought back up. From what I see in `TServerUtils`, this may only feasibly occur on a one TServer cluster as the logic to obtain the larger pool size will never be larger than the active pool count. 
   2021-05-21T07:44:01,916 [rpc.TServerUtils] INFO : Increasing server thread pool size on CompactionCoordinator to 1
   


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635582041



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -95,6 +117,8 @@
 
   private Supplier<Set<CompactionServiceId>> servicesInUse;
 
+  private Set<CompactionServiceId> servicesUsed = new ConcurrentSkipListSet<>();
+
   // status of special compactions
   private enum SpecialStatus {
     NEW, SELECTING, SELECTED, NOT_ACTIVE, CANCELED

Review comment:
       I like your suggestion, renamed in e81e790




-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-849952173


   On a test table in Uno w/ 2B entries and 128 tablets I was able to run 40 concurrent external compactions in AKS Kubernetes today (had lots of help from @dlmarion  w/ Kubernetes).  I am continuing to poke at this PR, but I think its looking pretty good.  I hope to run test on cluster+Kubernetes soon instead of Uno+Kubernetes.
   
   ```
   root@uno> listcompactions
    SERVER               | AGE       | TYPE  | REASON | READ  | WROTE | TABLE           | TABLET                                   | INPUT | OUTPUT                              | ITERATORS | ITERATOR OPTIONS
   172.20.128.185:9101 (ext) |   26s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9401a;92012                            |     1 |        /2/t-0000cj0/A0002ay3.rf_tmp |        [] | {}
   172.20.128.188:9101 (ext) |  23s856ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;ba01;b801                              |     1 |        /2/t-00017pv/A0002ayb.rf_tmp |        [] | {}
   172.20.128.41:9101 (ext) |  34s598ms |  FULL |   USER | 3.64M | 3.64M |       testTable | 2;0801b;0602                             |     1 |        /2/t-00004hf/A0002axd.rf_tmp |        [] | {}
   172.20.128.53:9101 (ext) |  35s340ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;02014<                                 |     1 |        /2/t-00017dz/A0002axb.rf_tmp |        [] | {}
   172.20.129.116:9101 (ext) |  23s812ms |  FULL |   USER | 2.40M | 2.40M |       testTable | 2;b801;b601                              |     1 |        /2/t-00004ij/A0002ayc.rf_tmp |        [] | {}
   172.20.129.13:9101 (ext) |   22s80ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b400b;b200b                            |     1 |        /2/t-0000cjc/A0002aye.rf_tmp |        [] | {}
   172.20.129.184:9101 (ext) |  25s695ms |  FULL |   USER | 2.79M | 2.79M |       testTable | 2;9801c;9602                             |     1 |        /2/t-00004gy/A0002ay7.rf_tmp |        [] | {}
   172.20.129.195:9101 (ext) |  34s670ms |  FULL |   USER | 3.30M | 3.30M |       testTable | 2;0401a;02014                            |     1 |        /2/t-0000cl4/A0002axc.rf_tmp |        [] | {}
   172.20.129.80:9101 (ext) |  28s262ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8c015;8a01e                            |     1 |        /2/t-0000cq8/A0002axv.rf_tmp |        [] | {}
   172.20.130.1:9101 (ext) |  24s517ms |  FULL |   USER | 2.51M | 2.51M |       testTable | 2;be00c;bc00e                            |     1 |        /2/t-00017xu/A0002ay9.rf_tmp |        [] | {}
   172.20.130.104:9101 (ext) |  23s609ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;b601;b400b                             |     1 |        /2/t-00017s0/A0002ayd.rf_tmp |        [] | {}
   172.20.130.113:9101 (ext) |  30s713ms |  FULL |   USER | 3.88M | 3.88M |       testTable | 2;4201;4                                 |     1 |        /2/t-00017kz/A0002axn.rf_tmp |        [] | {}
   172.20.130.201:9101 (ext) |  26s766ms |  FULL |   USER | 2.91M | 2.91M |       testTable | 2;9;8e00d                                |     1 |        /2/t-000000d/A0002ay0.rf_tmp |        [] | {}
   172.20.130.249:9101 (ext) |  28s350ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;46017;44014                            |     1 |        /2/t-00017xf/A0002axu.rf_tmp |        [] | {}
   172.20.131.143:9101 (ext) |  24s460ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;bc00e;ba01                             |     1 |        /2/t-0000cnb/A0002aya.rf_tmp |        [] | {}
   172.20.131.150:9101 (ext) |  25s101ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9a01f;9801c                            |     1 |        /2/t-00017qv/A0002ay8.rf_tmp |        [] | {}
   172.20.131.35:9101 (ext) |  26s969ms |  FULL |   USER | 2.77M | 2.77M |       testTable | 2;48016;46017                            |     1 |        /2/t-00004ho/A0002axz.rf_tmp |        [] | {}
   172.20.131.82:9101 (ext) |  33s715ms |  FULL |   USER | 3.36M | 3.36M |       testTable | 2;0c01a;0a01d                            |     1 |        /2/t-0000ctq/A0002axg.rf_tmp |        [] | {}
   172.20.132.14:9101 (ext) |  25s729ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4c01;4a018                             |     1 |        /2/t-0000col/A0002ay6.rf_tmp |        [] | {}
   172.20.132.141:9101 (ext) |  28s416ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8a01e;8801d                            |     1 |        /2/t-00017so/A0002axt.rf_tmp |        [] | {}
   172.20.132.187:9101 (ext) |  28s737ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8801d;86021                            |     1 |        /2/t-00004gp/A0002axs.rf_tmp |        [] | {}
   172.20.132.226:9101 (ext) |  30s934ms |  FULL |   USER | 4.30M | 4.30M |       testTable | 2;8;7e00d                                |     1 |        /2/t-0000000/A0002axl.rf_tmp |        [] | {}
   172.20.132.26:9101 (ext) |  29s994ms |  FULL |   USER | 3.17M | 3.17M |       testTable | 2;22016;2                                |     1 |        /2/t-00017lc/A0002axq.rf_tmp |        [] | {}
   172.20.133.113:9101 (ext) |  29s711ms |  FULL |   USER | 3.20M | 3.20M |       testTable | 2;44014;4201                             |     1 |        /2/t-0000clm/A0002axr.rf_tmp |        [] | {}
   172.20.133.144:9101 (ext) |  32s707ms |  FULL |   USER | 4.10M | 4.10M |       testTable | 2;1;0e013                                |     1 |        /2/t-0000008/A0002axi.rf_tmp |        [] | {}
   172.20.133.182:9101 (ext) |  28s313ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;8e00d;8c015                            |     1 |        /2/t-00017zq/A0002axw.rf_tmp |        [] | {}
   172.20.133.255:9101 (ext) |  26s703ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;92012;9                                |     1 |        /2/t-00017kk/A0002ay1.rf_tmp |        [] | {}
   172.20.133.55:9101 (ext) |  25s724ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;26024;2401d                            |     1 |        /2/t-00017us/A0002ay5.rf_tmp |        [] | {}
   172.20.134.114:9101 (ext) |  26s191ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4a018;48016                            |     1 |        /2/t-00017wj/A0002ay2.rf_tmp |        [] | {}
   172.20.134.139:9101 (ext) |  25s993ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;9602;9401a                             |     1 |        /2/t-00017rl/A0002ay4.rf_tmp |        [] | {}
   172.20.134.161:9101 (ext) |  30s202ms |  FULL |   USER | 3.70M | 3.70M |       testTable | 2;86021;84019                            |     1 |        /2/t-00017hh/A0002axp.rf_tmp |        [] | {}
   172.20.134.250:9101 (ext) |  30s186ms |  FULL |   USER | 3.19M | 3.19M |       testTable | 2;84019;82013                            |     1 |        /2/t-0000co6/A0002axo.rf_tmp |        [] | {}
   172.20.134.61:9101 (ext) |   34s46ms |  FULL |   USER | 4.52M | 4.52M |       testTable | 2;0602;0401a                             |     1 |        /2/t-00017mi/A0002axf.rf_tmp |        [] | {}
   172.20.135.166:9101 (ext) |   33s46ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;0e013;0c01a                            |     1 |        /2/t-0001816/A0002axh.rf_tmp |        [] | {}
   172.20.135.191:9101 (ext) |  32s423ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2;1e00e                                |     1 |        /2/t-0000005/A0002axj.rf_tmp |        [] | {}
   172.20.135.222:9101 (ext) |  34s165ms |  FULL |   USER | 4.43M | 4.43M |       testTable | 2;0a01d;0801b                            |     1 |        /2/t-00017gs/A0002axe.rf_tmp |        [] | {}
   172.20.135.41:9101 (ext) |  26s997ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;2401d;22016                            |     1 |        /2/t-0000cle/A0002axy.rf_tmp |        [] | {}
   172.20.136.109:9101 (ext) |  27s177ms |  FULL |   USER | 2.98M | 2.98M |       testTable | 2;1201;1                                 |     1 |        /2/t-00017v7/A0002axx.rf_tmp |        [] | {}
   172.20.136.162:9101 (ext) |  31s305ms |  FULL |   USER | 3.07M | 3.07M |       testTable | 2;4;3e00f                                |     1 |        /2/t-0000002/A0002axk.rf_tmp |        [] | {}
   172.20.136.52:9101 (ext) |  30s742ms |  FULL |   USER | 3.35M | 3.35M |       testTable | 2;82013;8                                |     1 |        /2/t-00017iu/A0002axm.rf_tmp |        [] | {}
   ```


-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   I was able to successfully run the example @dlmarion gave on the website and it is pretty cool! I was also able to catch an entry in the metadata:
   <pre>
   4< ecomp:ECID:04d6f4a3-3caa-46e6-9ea5-ce77a0ddc8dc []	{"inputs":["hdfs://localhost:8020/accumulo/tables/4/default_tablet/A000005v.rf"],"nextFiles":[],"tmp":"hdfs://localhost:8020/accumulo/tables/4/default_tablet/A000005y.rf_tmp","dest":"hdfs://localhost:8020/accumulo/tables/4/default_tablet/A000005y.rf","compactor":"ip-10-113-12-25:9101","kind":"USER","executorId":"DCQ1","priority":288230376151711745,"propDels":false,"selectedAll":true,"compactionId":2}
   </pre>
   I love how everything about the compaction is explicit AND in one place. I think this will be invaluable for debugging. 


-- 
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.

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



[GitHub] [accumulo] Manno15 commented on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-845899886


   I also noticed that this log message below can get chatty in the right scenario (one Tserver cluster that has the Tserver crash or killed). In that scenario, it will repeat this log message until one TServer is brought back up. From what I see in `TServerUtils`, this may only feasibly occur on a one TServer cluster as the logic to obtain the larger pool size will never be larger than the active pool count. 
   2021-05-21T07:44:01,916 [rpc.TServerUtils] INFO : Increasing server thread pool size on CompactionCoordinator to 1
   


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632109671



##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
##########
@@ -204,6 +210,54 @@ public boolean testClassLoad(final String className, final String asTypeName)
     }
   }
 
+  @Override
+  public List<ActiveCompaction> getActiveCompactions()
+      throws AccumuloException, AccumuloSecurityException {
+
+    List<HostAndPort> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    List<String> tservers = getTabletServers();
+
+    int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
+    var executorService =
+        ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions", false);
+    try {
+      List<Future<List<ActiveCompaction>>> futures = new ArrayList<>();
+
+      for (String tserver : tservers) {
+        futures.add(executorService.submit(() -> getActiveCompactions(tserver)));
+      }
+
+      for (HostAndPort compactorAddr : compactors) {
+        futures
+            .add(
+                executorService
+                    .submit(
+                        () -> ExternalCompactionUtil.getActiveCompaction(compactorAddr, context)
+                            .stream().map(tac -> new ActiveCompactionImpl(context, tac,
+                                compactorAddr, CompactionHost.Type.COMPACTOR))
+                            .collect(Collectors.toList())));

Review comment:
       Improved this a bit in 162dd806b1754f890c4f139a3611c54024c072a2.  Tried to rename the var `tac` to `thriftActiveCompaction` but it caused the formatting of the updated code to be unsightly.




-- 
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.

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



[GitHub] [accumulo] ctubbsii commented on pull request #2096: External Compactions

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






-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635581737



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
##########
@@ -277,6 +378,130 @@ private void checkIfUserCompactionCanceled() {
     }
   }
 
+  private void initializeSelection(

Review comment:
       @milleruntime  I added a comment in e81e790.  Let me know if that looks good.




-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   > @milleruntime looking at your comment I opened #2110. This PR is so large, I think we can use the external compaction project to open issue and not just rely on the comment in this PR. That opens the possibility of merging this PR and doing some of those issues as follow on PRs.
   
   That works since I don't anticipate seeing something that would prevent you guys from merging. I will just keep putting comments here against the code in the PR since that is easier for me. They could all be follow on work if that's easier for you guys.


-- 
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.

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



[GitHub] [accumulo] milleruntime commented on pull request #2096: External Compactions

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


   > Another option would be to click the "Viewed" checkbox on all of the thrift files, then they would be removed from your view. Not sure if that persists across browser sessions or not.
   
   I think this will only hide the .thrift files (which I want to see). It won't hide the generated Java files.


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633717986



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       So, it looks like the CompactionFinalizer scans the Metadata table to determine which TabletServer to notify when a compaction fails or succeeds. We might be able to remove this after all.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632733581



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionJobImpl.java
##########
@@ -40,14 +42,25 @@
   private final Set<CompactableFile> files;
   private final CompactionKind kind;
   private boolean selectedAll;
+  private boolean hasSelectedAll;
 
-  CompactionJobImpl(long priority, CompactionExecutorId executor, Collection<CompactableFile> files,
-      CompactionKind kind, boolean selectedAllFiles) {
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind, boolean selectedAllFiles) {
     this.priority = priority;
     this.executor = Objects.requireNonNull(executor);
     this.files = Set.copyOf(files);
-    this.kind = kind;
+    this.kind = Objects.requireNonNull(kind);
     this.selectedAll = selectedAllFiles;
+    this.hasSelectedAll = true;
+  }
+
+  public CompactionJobImpl(long priority, CompactionExecutorId executor,
+      Collection<CompactableFile> files, CompactionKind kind) {
+    this.priority = priority;
+    this.executor = Objects.requireNonNull(executor);
+    this.files = Set.copyOf(files);
+    this.kind = Objects.requireNonNull(kind);
+    this.hasSelectedAll = false;

Review comment:
       I will look into 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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   >  I noticed with the CompactionCoordinator and CompactionFinalizer both have filesize and entries parameters but they are in the opposite order as each other.
   
   Consistency is nice.  Looking the message you posted, thinking we should also minimize the INFO message per compaction in the coordinator like the Compactor.


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634305939



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =

Review comment:
       I'm going to look at making this change.




-- 
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.

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



[GitHub] [accumulo] dlmarion merged pull request #2096: External Compactions

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


   


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633728857



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       Agreed, I'm going to push another commit shortly that will significantly reduce the lines of code for the re-start and remove the getTServer method from RunningCompaction.




-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842406924


   > I'm wondering if there are ways this could be made a bit more modular and the configuration a bit more explicit, so that the default compaction service isn't so monolithic, and users can very explicitly swap out the external executor implementation for another one. 
   
   In the current design tablets and tservers are authorities, within a distributed system, on what external compactions are running and queued.  There are RPCs to reserve, fail, and commit external compactions within the tserver AND there is an RPC to get summary information about external queues in the tserver.  These RPCs are structured specifically for the way the rest of the distributed system (coordinator and compactors) works.  Should these tserver RPCs be generalized?  If so, how?


-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844378288


   @Manno15 I think I may have fixed the issue you were seeing in 3095d3d, if line 1055 was the !closed check.  That was an incorrect sanity check, looking at how the close() method works, closed could be true there and that would be ok.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   @Manno15 the feedback on logging was great.  I made some quick changes with the following goals.
   
    * At INFO when things are working normally there should only be  two INFO message per compaction. One message before and one after.
    * At DEBUG there are multiple messages per compaction giving more detailed information about compactions that are working as expected.
    * At TRACE there are only message that people would want to see when there is a suspected bug in the compactor code or performance problem.  These are messages that people would not want to see whenever the system is working as expected.
   
   I Am not sure if I achieved the INFO goal of two message per compaction because I didn't try running it.
   
   


-- 
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.

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



[GitHub] [accumulo] Manno15 edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844253910


   One other very minor thing (and I do mean very minor; feel free to ignore this). I noticed with the `CompactionCoordinator` and `CompactionFinalizer` both have filesize and entries parameters but they are in the opposite order as each other. When reading the logs, it would be more consistent if they matched (since they come one after another in the logs). The screenshot below shows the log messages I am referring to.
   
   ![image](https://user-images.githubusercontent.com/29436247/118846534-ce447780-b89a-11eb-81c3-8075ff14c20d.png)
   


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r632072336



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.compaction;
+
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {

Review comment:
       > why make it more awkward right now if we don't have to?
   
   None of those methods are needed to write a compaction planner plugin, they are only used internally by the implementation outside of the realm of SPI interfaces.  So exposing them in the SPI could make changing the implementation harder 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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > I was able to successfully run the example @dlmarion gave on the website and it is pretty cool! I was also able to catch an entry in the metadata:
   
   @milleruntime looking at your comment I opened #2110. This PR is so large, I think we can use the external compaction project to open issue and not just rely on the comment in this PR.  That opens the possibility of merging this PR and doing some of those issues as follow on PRs.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r631958261



##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.compaction;
+
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {

Review comment:
       I was trying to avoid putting all of the external/internal method in the SPI 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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r637241473



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/metrics/CompactionExecutorsMetrics.java
##########
@@ -77,10 +92,47 @@ public synchronized AutoCloseable addExecutor(CompactionExecutorId ceid,
 
   @Override
   public void prepareMetrics() {
+
+    if (externalMetricsSupplier != null) {
+
+      Set<CompactionExecutorId> seenIds = new HashSet<>();
+
+      MetricsRegistry registry = super.getRegistry();
+
+      synchronized (exMetrics) {
+        externalMetricsSupplier.get().forEach(ecm -> {
+          seenIds.add(ecm.ceid);
+
+          ExMetrics exm = exMetrics.computeIfAbsent(ecm.ceid, id -> {
+            ExMetrics m = new ExMetrics();
+            m.queuedGauge = registry.newGauge(id.canonical().replace('.', '_') + "_queued",

Review comment:
       Wondering if a queue being configured out of existence causes any problems for the gauges that may be left around. 




-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842584575


   Omitting a lot of detail, the following are the basics of what the current system does.  The first step uses a pluggable planner.
   
   1. Plan compaction :  Take current tablets files, current running compactions for the tablet, and emit set of compactions jobs.  A compaction job is set of files to compact, a  priority, and a destination queue.
    2. Queue Jobs : An attempt to queue the jobs emitted from the planner is made by doing the following.
       1. Cancel anything that was previously queued by the planner for the tablet.  If this fails, go back to planning step and try again because things probably changed during planning.
       2. Queue the new job from the planner on the desired priority queues
    3. When an internal or external compactor thread finishes a task, it takes the next highest priority job from the queue and actually runs a compaction.
   
   I don' think interface `BiFunction<Set<File>, CompactionConfig, Future<File>>` would be sufficient to achieve this functionality.  It would not support the current functionality of priority queues and canceling queued work when things change (new files arrive, job start/finish during planning). In the current impl the priority queue for local compactions is very precise.  For external compaction there is essentially a global priority queue that is approximate (eventually consistent) that may not always start the highest priority job next, but usually will.
   
   


-- 
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.

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



[GitHub] [accumulo] milleruntime commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r631144716



##########
File path: core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperations.java
##########
@@ -112,6 +115,14 @@ void setProperty(final String property, final String value)
   List<ActiveCompaction> getActiveCompactions(String tserver)
       throws AccumuloException, AccumuloSecurityException;
 
+  /**
+   * List all compactions running in Accumulo

Review comment:
       Should mention this will return internal and external compactions.

##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),

Review comment:
       Should start description with a capital letter since this will be put on the generated web page for Configuration.

##########
File path: core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java
##########
@@ -204,6 +210,54 @@ public boolean testClassLoad(final String className, final String asTypeName)
     }
   }
 
+  @Override
+  public List<ActiveCompaction> getActiveCompactions()
+      throws AccumuloException, AccumuloSecurityException {
+
+    List<HostAndPort> compactors = ExternalCompactionUtil.getCompactorAddrs(context);
+    List<String> tservers = getTabletServers();
+
+    int numThreads = Math.max(4, Math.min((tservers.size() + compactors.size()) / 10, 256));
+    var executorService =
+        ThreadPools.createFixedThreadPool(numThreads, "getactivecompactions", false);
+    try {
+      List<Future<List<ActiveCompaction>>> futures = new ArrayList<>();
+
+      for (String tserver : tservers) {
+        futures.add(executorService.submit(() -> getActiveCompactions(tserver)));
+      }
+
+      for (HostAndPort compactorAddr : compactors) {
+        futures
+            .add(
+                executorService
+                    .submit(
+                        () -> ExternalCompactionUtil.getActiveCompaction(compactorAddr, context)
+                            .stream().map(tac -> new ActiveCompactionImpl(context, tac,
+                                compactorAddr, CompactionHost.Type.COMPACTOR))
+                            .collect(Collectors.toList())));

Review comment:
       I think some creative renaming of variables and/or using a static imports could clean up the formatting here.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/ExternalCompactionUtil.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.util.compaction;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.clientImpl.ClientContext;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ZooReader;
+import org.apache.accumulo.fate.zookeeper.ZooSession;
+import org.apache.thrift.TException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExternalCompactionUtil {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExternalCompactionUtil.class);
+
+  /**
+   * Utility for returning the address of a service in the form host:port
+   *
+   * @param address
+   *          HostAndPort of service
+   * @return host and port
+   */
+  public static String getHostPortString(HostAndPort address) {
+    if (address == null) {
+      return null;
+    }
+    return address.getHost() + ":" + address.getPort();

Review comment:
       I think the `toString()` of our `HostAndPort` in core util already does this.

##########
File path: core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionExecutorIdImpl.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.util.compaction;
+
+import org.apache.accumulo.core.spi.compaction.CompactionExecutorId;
+import org.apache.accumulo.core.spi.compaction.CompactionServiceId;
+
+import com.google.common.base.Preconditions;
+
+public class CompactionExecutorIdImpl extends CompactionExecutorId {

Review comment:
       I assume this class was created to be separate from the SPI but I am not seeing anything internal exposed. It seems like it could just be included in the super class.

##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       I am not sure if all of the THRIFTCLIENT properties that start with "coordinator.server.thrift" need the word "thrift" in the property. It seems like they could be shortened. For example, I think this one could be "coordinator.server.message.size.max" instead. I am also not sure if THRIFTCLIENT makes sense in the enum names. 

##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),

Review comment:
       Begin description with capital letter.




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633903466



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =

Review comment:
       That could work w/ some slight tweaks.  The compactionFailed method in the coordinator is kinda thrifty and only takes a single ecid.  Could have package private non thrifty compactionFailed method in the coordinator that takes a collection of ecids and is called by the dead compaction detector and the thrifty compactionFailed method.




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634728050



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -313,6 +291,61 @@ public void run() {
     LOG.info("Shutting down");
   }
 
+  private void updateSummaries() {
+    ExecutorService executor =
+        ThreadPools.createFixedThreadPool(10, "Compaction Summary Gatherer", false);

Review comment:
       Should the number of threads (10 here) be some factor (like 10%) of the # of tservers in tServerSet?




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   @Manno15 re the consistency of those fields, that data was coming from a thrift class and GSON generator json, so not sure they could reliably be ordered consistently.


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on pull request #2096: External Compactions

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


   > My understanding of the original idea to create an "external" compactions, was to create them "external" to Accumulo...
   
   That was never an explicit goal I had. To achieve this goal, thinking about the following.  Currently the implementation of the coordinator and compactors components use a lot of internal Accumulo code.  To move these to external repo would require refactoring them to use only API and SPI calls into Accumulo. To do this would require exposing a lot internal low level API/SPI calls related to compactions(not sure what these would be, that would need exploration).   For this exercise to be useful these newly exposed API/SPI calls would have to be generally useful and not overfit (meaning the new API/SPI calls can only be used to implement the current design).  To avoid overfitting, it seems we would need to have multiple other designs in mind that we could test against these new API/SPI calls to verify that we have not overfit.  Is there another path other than what I have described?  Does this path actually lower the complexity? I am not sure it would, I think the answer to that de
 pends on what these new API/SPI calls are.
   


-- 
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.

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



[GitHub] [accumulo] Manno15 commented on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-845856835


   @keith-turner @dlmarion is there any particular aspect or feature that you think needs to be tested specifically? I've done multiple runs with the normal testing procedures (multiple tservers, agitation, etc) and haven't run into any issues so far. 


-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842394039


   >  it seems that the way the external compactions are exposed through to the user in the compaction configuration is through the specially named externalQueue, which seems to be a special feature of the default compaction service. Is that correct?
   
   Its a feature of the default compaction planner, not the default compaction service.  Each compaction service has a configured planner that decides what to compact where.  The [default compaction planner](https://github.com/apache/accumulo/blob/3983cd23629789c1ed0020d01fdd968eabfe590f/core/src/main/java/org/apache/accumulo/core/spi/compaction/DefaultCompactionPlanner.java#L73) accepts the config externalQueue.  It then takes this information and calls the SPI method [getExternalExecutor](https://github.com/apache/accumulo/blob/3983cd23629789c1ed0020d01fdd968eabfe590f/core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java#L38).  Anyone could write a compaction planner that uses external queues by calling this same SPI method.  The planners configuration could work any way the author likes.


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633441752



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       Working on this.

##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -1022,6 +1022,49 @@
   REPLICATION_RPC_TIMEOUT("replication.rpc.timeout", "2m", PropertyType.TIMEDURATION,
       "Amount of time for a single replication RPC call to last before failing"
           + " the attempt. See replication.work.attempts."),
+  // Compactor properties
+  COMPACTOR_PREFIX("compactor.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compactor server."),
+  COMPACTOR_PORTSEARCH("compactor.port.search", "true", PropertyType.BOOLEAN,
+      "if the compactor.port.client is in use, search higher ports until one is available"),
+  COMPACTOR_CLIENTPORT("compactor.port.client", "9101", PropertyType.PORT,
+      "The port used for handling client connections on the compactor servers"),
+  COMPACTOR_MINTHREADS("compactor.server.threads.minimum", "1", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests."),
+  COMPACTOR_MINTHREADS_TIMEOUT("compactor.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COMPACTOR_THREADCHECK("compactor.server.threadcheck.time", "1s", PropertyType.TIMEDURATION,
+      "The time between adjustments of the server thread pool."),
+  COMPACTOR_MAX_MESSAGE_SIZE("compactor.server.message.size.max", "10M", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a tablet server."),
+  // CompactionCoordinator properties
+  COORDINATOR_PREFIX("coordinator.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the accumulo compaction coordinator server."),
+  COORDINATOR_THRIFTCLIENT_PORTSEARCH("coordinator.thrift.port.search", "false",
+      PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available"),
+  COORDINATOR_THRIFTCLIENT_CLIENTPORT("coordinator.thrift.port.client", "9100", PropertyType.PORT,
+      "The port used for handling Thrift client connections on the compaction coordinator server"),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS("coordinator.server.thrift.threads.minimum", "1",
+      PropertyType.COUNT, "The minimum number of threads to use to handle incoming requests."),
+  COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT("coordinator.server.thrift.threads.timeout", "0s",
+      PropertyType.TIMEDURATION,
+      "The time after which incoming request threads terminate with no work available.  Zero (0) will keep the threads alive indefinitely."),
+  COORDINATOR_THRIFTCLIENT_THREADCHECK("coordinator.server.thrift.threadcheck.time", "1s",
+      PropertyType.TIMEDURATION, "The time between adjustments of the server thread pool."),
+  COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE("coordinator.server.thrift.message.size.max", "10M",

Review comment:
       Resolved in 011a861832036fd22d0a8ee55e4daa2e11c90801

##########
File path: server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.server.compaction;
+
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RetryableThriftCall<T> {
+
+  public static class RetriesExceededException extends Exception {
+
+    private static final long serialVersionUID = 1L;
+
+    public RetriesExceededException() {
+      super();
+    }
+
+    public RetriesExceededException(String message, Throwable cause, boolean enableSuppression,
+        boolean writableStackTrace) {
+      super(message, cause, enableSuppression, writableStackTrace);
+    }
+
+    public RetriesExceededException(String message, Throwable cause) {
+      super(message, cause);
+    }
+
+    public RetriesExceededException(String message) {
+      super(message);
+    }
+
+    public RetriesExceededException(Throwable cause) {
+      super(cause);
+    }
+
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(RetryableThriftCall.class);
+  public static final long MAX_WAIT_TIME = 60000;
+
+  private final long start;
+  private final long maxWaitTime;
+  private int maxNumRetries;
+  private final RetryableThriftFunction<T> function;
+  private final boolean retryForever;
+
+  /**
+   * RetryableThriftCall constructor
+   *
+   * @param start
+   *          initial wait time
+   * @param maxWaitTime
+   *          max wait time
+   * @param maxNumRetries
+   *          number of times to retry, 0 to retry forever
+   * @param function
+   *          function to execute
+   */
+  public RetryableThriftCall(long start, long maxWaitTime, int maxNumRetries,
+      RetryableThriftFunction<T> function) {
+    this.start = start;
+    this.maxWaitTime = maxWaitTime;
+    this.maxNumRetries = maxNumRetries;
+    this.function = function;
+    this.retryForever = (maxNumRetries == 0);
+  }
+
+  /**
+   * Attempts to call the function, waiting and retrying when TException is thrown. Wait time is
+   * initially set to the start time and doubled each time, up to the maximum wait time. If
+   * maxNumRetries is 0, then this will retry forever. If maxNumRetries is non-zero, then a
+   * RuntimeException is thrown when it has exceeded he maxNumRetries parameter.
+   *
+   * @return T
+   * @throws RetriesExceededException
+   *           when maximum number of retries has been exceeded and the cause is set to the last
+   *           TException
+   */
+  public T run() throws RetriesExceededException {
+    long waitTime = start;

Review comment:
       Resolved in 3983cd23629789c1ed0020d01fdd968eabfe590f

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       I guess the question is whether or not the CompactionCoordinator can get stale data from the metadata table. The code tries to find the TabletServer running this compaction. It's possible that the Tablet could be in the process of being re-hosted (in the case of a TServer crash). In the current implementation an entry would not be made in the internal running list because we didn't find the TServer. If we just use the Metadata entries, we could put incorrect information in the running list.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }
+
+          // As a fallback, try them all
+          if (!matchFound) {
+            LOG.debug("Checking all tservers for external running compaction, extent: {}", extent);
+            for (TServerInstance tsi : tservers) {
+              TabletClientService.Client client = null;
+              try {
+                client = getTabletServerConnection(tsi);
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                }
+              } catch (TException e) {
+                LOG.error(
+                    "Error from tserver {} while trying to check if external compaction is running, trying next tserver",
+                    ExternalCompactionUtil.getHostPortString(tsi.getHostAndPort()), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            }
+          }
+
+          if (!matchFound) {
+            LOG.warn(
+                "There is an external compaction running on a compactor, but could not find corresponding tablet server. Extent: {}, Compactor: {}, Compaction: {}",
+                extent, hp, job);
+          }
+        });
+      }
+      tservers.clear();
+    } else {
+      LOG.info("No running tablet servers found, continuing startup");
+    }
+
+    tserverSet.startListeningForTabletServerChanges();
+    startDeadCompactionDetector();
+
+    LOG.info("Starting loop to check tservers for compaction summaries");
+    while (!shutdown) {
+      long start = System.currentTimeMillis();
+      tserverSet.getCurrentServers().forEach(tsi -> {
+        try {
+          TabletClientService.Client client = null;
+          try {
+            LOG.debug("Contacting tablet server {} to get external compaction summaries",
+                tsi.getHostPort());
+            client = getTabletServerConnection(tsi);
+            List<TCompactionQueueSummary> summaries =
+                client.getCompactionQueueInfo(TraceUtil.traceInfo(), getContext().rpcCreds());
+            summaries.forEach(summary -> {
+              QueueAndPriority qp =
+                  QueueAndPriority.get(summary.getQueue().intern(), summary.getPriority());
+              synchronized (qp) {
+                TIME_COMPACTOR_LAST_CHECKED.computeIfAbsent(qp.getQueue(), k -> 0L);
+                QUEUE_SUMMARIES.update(tsi, summaries);
+              }
+            });
+          } finally {
+            ThriftUtil.returnClient(client);
+          }
+        } catch (TException e) {
+          LOG.warn("Error getting external compaction summaries from tablet server: {}",
+              tsi.getHostAndPort(), e);
+          QUEUE_SUMMARIES.remove(Set.of(tsi));
+        }
+      });
+
+      long now = System.currentTimeMillis();
+      TIME_COMPACTOR_LAST_CHECKED.forEach((k, v) -> {
+        if ((now - v) > getMissingCompactorWarningTime()) {
+          LOG.warn("No compactors have checked in with coordinator for queue {} in {}ms", k,
+              getMissingCompactorWarningTime());
+        }
+      });
+
+      long checkInterval = getTServerCheckInterval();
+      long duration = (System.currentTimeMillis() - start);
+      if (checkInterval - duration > 0) {
+        LOG.debug("Waiting {}ms for next tserver check", (checkInterval - duration));
+        UtilWaitThread.sleep(checkInterval - duration);
+      }
+    }
+
+    LOG.info("Shutting down");
+  }
+
+  protected void startDeadCompactionDetector() {
+    new DeadCompactionDetector(getContext(), compactionFinalizer, schedExecutor).start();
+  }
+
+  protected long getMissingCompactorWarningTime() {
+    return FIFTEEN_MINUTES;
+  }
+
+  protected long getTServerCheckInterval() {
+    return this.aconf.getTimeInMillis(Property.COORDINATOR_TSERVER_COMPACTION_CHECK_INTERVAL);
+  }
+
+  protected TabletMetadata getMetadataEntryForExtent(KeyExtent extent) {
+    return getContext().getAmple().readTablets().forTablet(extent)
+        .fetch(ColumnType.LOCATION, ColumnType.PREV_ROW).build().stream().findFirst().orElse(null);
+  }
+
+  /**
+   * Callback for the LiveTServerSet object to update current set of tablet servers, including ones
+   * that were deleted and added
+   *
+   * @param current
+   *          current set of live tservers
+   * @param deleted
+   *          set of tservers that were removed from current since last update
+   * @param added
+   *          set of tservers that were added to current since last update
+   */
+  @Override
+  public void update(LiveTServerSet current, Set<TServerInstance> deleted,
+      Set<TServerInstance> added) {
+
+    // run() will iterate over the current and added tservers and add them to the internal
+    // data structures. For tservers that are deleted, we need to remove them from QUEUES
+    // and INDEX
+    QUEUE_SUMMARIES.remove(deleted);
+  }
+
+  /**
+   * Return the next compaction job from the queue to a Compactor
+   *
+   * @param queueName
+   *          queue
+   * @param compactorAddress
+   *          compactor address
+   * @return compaction job
+   */
+  @Override
+  public TExternalCompactionJob getCompactionJob(TInfo tinfo, TCredentials credentials,
+      String queueName, String compactorAddress, String externalCompactionId) throws TException {
+
+    // do not expect users to call this directly, expect compactors to call this method
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+    final String queue = queueName.intern();
+    LOG.debug("getCompactionJob called for queue {} by compactor {}", queue, compactorAddress);
+    TIME_COMPACTOR_LAST_CHECKED.put(queue, System.currentTimeMillis());
+
+    TExternalCompactionJob result = null;
+
+    PrioTserver prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+
+    while (prioTserver != null) {
+      TServerInstance tserver = prioTserver.tserver;
+
+      LOG.debug("Getting compaction for queue {} from tserver {}", queue, tserver.getHostAndPort());
+      // Get a compaction from the tserver
+      TabletClientService.Client client = null;
+      try {
+        client = getTabletServerConnection(tserver);
+        TExternalCompactionJob job =
+            client.reserveCompactionJob(TraceUtil.traceInfo(), getContext().rpcCreds(), queue,
+                prioTserver.prio, compactorAddress, externalCompactionId);
+        if (null == job.getExternalCompactionId()) {
+          LOG.debug("No compactions found for queue {} on tserver {}, trying next tserver", queue,
+              tserver.getHostAndPort(), compactorAddress);
+
+          QUEUE_SUMMARIES.removeSummary(tserver, queueName, prioTserver.prio);
+          prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+          continue;
+        }
+        RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+            new RunningCompaction(job, compactorAddress, tserver));
+        LOG.debug("Returning external job {} to {}", job.externalCompactionId, compactorAddress);
+        result = job;
+        break;
+      } catch (TException e) {
+        LOG.warn("Error from tserver {} while trying to reserve compaction, trying next tserver",
+            ExternalCompactionUtil.getHostPortString(tserver.getHostAndPort()), e);
+        QUEUE_SUMMARIES.removeSummary(tserver, queueName, prioTserver.prio);
+        prioTserver = QUEUE_SUMMARIES.getNextTserver(queueName);
+      } finally {
+        ThriftUtil.returnClient(client);
+      }
+    }
+
+    if (result == null) {
+      LOG.debug("No tservers found for queue {}, returning empty job to compactor {}", queue,
+          compactorAddress);
+      result = new TExternalCompactionJob();
+    }
+
+    return result;
+
+  }
+
+  /**
+   * Return the Thrift client for the TServer
+   *
+   * @param tserver
+   *          tserver instance
+   * @return thrift client
+   * @throws TTransportException
+   *           thrift error
+   */
+  protected TabletClientService.Client getTabletServerConnection(TServerInstance tserver)
+      throws TTransportException {
+    TServerConnection connection = tserverSet.getConnection(tserver);
+    TTransport transport =
+        ThriftTransportPool.getInstance().getTransport(connection.getAddress(), 0, getContext());
+    return ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
+  }
+
+  /**
+   * Return the Thrift client for the Compactor
+   *
+   * @param compactorAddress
+   *          compactor address
+   * @return thrift client
+   * @throws TTransportException
+   *           thrift error
+   */
+  protected Compactor.Client getCompactorConnection(HostAndPort compactorAddress)
+      throws TTransportException {
+    TTransport transport =
+        ThriftTransportPool.getInstance().getTransport(compactorAddress, 0, getContext());
+    return ThriftUtil.createClient(new Compactor.Client.Factory(), transport);
+  }
+
+  /**
+   * Called by the TabletServer to cancel the running compaction.
+   *
+   * @param tinfo
+   *          trace info
+   * @param credentials
+   *          tcredentials object
+   * @param externalCompactionId
+   *          compaction id
+   * @throws TException
+   *           thrift error
+   */
+  @Override
+  public void cancelCompaction(TInfo tinfo, TCredentials credentials, String externalCompactionId)

Review comment:
       Resolved in cdfec2afccd22ea344ad8fd3a3ce622ee5b87324

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       So, it looks like the CompactionFinalizer scans the Metadata table to determine which TabletServer to notify when a compaction fails or succeeds. We might be able to remove this after all.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       Agreed, I'm going to push another commit shortly that will significantly reduce the lines of code for the re-start and remove the getTServer method from RunningCompaction.

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =
+      new ConcurrentHashMap<>();
+
+  /* Map of queue name to last time compactor called to get a compaction job */
+  private static final Map<String,Long> TIME_COMPACTOR_LAST_CHECKED = new ConcurrentHashMap<>();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  protected SecurityOperation security;
+  protected final AccumuloConfiguration aconf;
+  protected CompactionFinalizer compactionFinalizer;
+  protected LiveTServerSet tserverSet;
+
+  private ServiceLock coordinatorLock;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private ScheduledThreadPoolExecutor schedExecutor;
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args) {
+    super("compaction-coordinator", opts, args);
+    aconf = getConfiguration();
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionCoordinator(ServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compaction-coordinator", opts, args);
+    aconf = conf;
+    schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    compactionFinalizer = createCompactionFinalizer(schedExecutor);
+    tserverSet = createLiveTServerSet();
+    setupSecurity();
+    startGCLogger(schedExecutor);
+    printStartupMsg();
+    startCompactionCleaner(schedExecutor);
+  }
+
+  protected CompactionFinalizer
+      createCompactionFinalizer(ScheduledThreadPoolExecutor schedExecutor) {
+    return new CompactionFinalizer(getContext(), schedExecutor);
+  }
+
+  protected LiveTServerSet createLiveTServerSet() {
+    return new LiveTServerSet(getContext(), this);
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  private void startCompactionCleaner(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> cleanUpCompactors(), 0, 5, TimeUnit.MINUTES);
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this CompactionCoordinator
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void getCoordinatorLock(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+    LOG.info("trying to get coordinator lock");
+
+    final String coordinatorClientAddress = ExternalCompactionUtil.getHostPortString(clientAddress);
+    final String lockPath = getContext().getZooKeeperRoot() + Constants.ZCOORDINATOR_LOCK;
+    final UUID zooLockUUID = UUID.randomUUID();
+
+    while (true) {
+
+      CoordinatorLockWatcher coordinatorLockWatcher = new CoordinatorLockWatcher();
+      coordinatorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+          ServiceLock.path(lockPath), zooLockUUID);
+      coordinatorLock.lock(coordinatorLockWatcher, coordinatorClientAddress.getBytes());
+
+      coordinatorLockWatcher.waitForChange();
+      if (coordinatorLockWatcher.isAcquiredLock()) {
+        break;
+      }
+      if (!coordinatorLockWatcher.isFailedToAcquireLock()) {
+        throw new IllegalStateException("manager lock in unknown state");
+      }
+      coordinatorLock.tryToCancelAsyncLockOrUnlock();
+
+      sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  /**
+   * Start this CompactionCoordinator thrift service to handle incoming client requests
+   *
+   * @return address of this CompactionCoordinator client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCoordinatorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, CompactionCoordinator.class,
+          getConfiguration());
+    }
+    final org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<
+        Iface> processor =
+            new org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Processor<>(
+                rpcProxy);
+    Property maxMessageSizeProperty =
+        (aconf.get(Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE) != null
+            ? Property.COORDINATOR_THRIFTCLIENT_MAX_MESSAGE_SIZE
+            : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COORDINATOR_THRIFTCLIENT_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COORDINATOR_THRIFTCLIENT_PORTSEARCH,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS,
+        Property.COORDINATOR_THRIFTCLIENT_MINTHREADS_TIMEOUT,
+        Property.COORDINATOR_THRIFTCLIENT_THREADCHECK, maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  @Override
+  public void run() {
+
+    ServerAddress coordinatorAddress = null;
+    try {
+      coordinatorAddress = startCoordinatorClientService();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the coordinator service", e1);
+    }
+    final HostAndPort clientAddress = coordinatorAddress.address;
+
+    try {
+      getCoordinatorLock(clientAddress);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IllegalStateException("Exception getting Coordinator lock", e);
+    }
+
+    // On a re-start of the coordinator it's possible that external compactions are in-progress.
+    // Attempt to get the running compactions on the compactors and then resolve which tserver
+    // the external compaction came from to re-populate the RUNNING collection.
+    LOG.info("Checking for running external compactions");
+    tserverSet.scanServers();
+    final Set<TServerInstance> tservers = tserverSet.getCurrentServers();
+    if (null != tservers && !tservers.isEmpty()) {
+      // On re-start contact the running Compactors to try and seed the list of running compactions
+      Map<HostAndPort,TExternalCompactionJob> running =
+          ExternalCompactionUtil.getCompactionsRunningOnCompactors(getContext());
+      if (running.isEmpty()) {
+        LOG.info("No compactions running on Compactors.");
+      } else {
+        LOG.info("Found {} running external compactions", running.size());
+        running.forEach((hp, job) -> {
+          // Find the tserver that has this compaction id
+          boolean matchFound = false;
+
+          // Attempt to find the TServer hosting the tablet based on the metadata table
+          // TODO use #1974 for more efficient metadata reads
+          KeyExtent extent = KeyExtent.fromThrift(job.getExtent());
+          LOG.debug("Getting tablet metadata for extent: {}", extent);
+          TabletMetadata tabletMetadata = getMetadataEntryForExtent(extent);
+
+          if (tabletMetadata != null && tabletMetadata.getExtent().equals(extent)
+              && tabletMetadata.getLocation() != null
+              && tabletMetadata.getLocation().getType() == LocationType.CURRENT) {
+
+            TServerInstance tsi = tservers.stream()
+                .filter(
+                    t -> t.getHostAndPort().equals(tabletMetadata.getLocation().getHostAndPort()))
+                .findFirst().orElse(null);
+
+            if (null != tsi) {
+              TabletClientService.Client client = null;
+              try {
+                LOG.debug(
+                    "Checking to see if tserver {} is running external compaction for extent: {}",
+                    tsi.getHostAndPort(), extent);
+                client = getTabletServerConnection(tsi);
+                boolean tserverMatch = client.isRunningExternalCompaction(TraceUtil.traceInfo(),
+                    getContext().rpcCreds(), job.getExternalCompactionId(), job.getExtent());
+                if (tserverMatch) {
+                  LOG.debug(
+                      "Tablet server {} is running external compaction for extent: {}, adding to running list",
+                      tsi.getHostAndPort(), extent);
+                  RUNNING.put(ExternalCompactionId.of(job.getExternalCompactionId()),
+                      new RunningCompaction(job, ExternalCompactionUtil.getHostPortString(hp),
+                          tsi));
+                  matchFound = true;
+                } else {
+                  LOG.debug("Tablet server {} is NOT running external compaction for extent: {}",
+                      tsi.getHostAndPort(), extent);
+                }
+              } catch (TException e) {
+                LOG.warn("Failed to notify tserver {}",
+                    tabletMetadata.getLocation().getHostAndPort(), e);
+              } finally {
+                ThriftUtil.returnClient(client);
+              }
+            } else {
+              LOG.info("Tablet server {} is not currently in live tserver set",
+                  tabletMetadata.getLocation().getHostAndPort());
+            }
+          } else {
+            LOG.info("No current location for extent: {}", extent);
+          }

Review comment:
       resolved in 90e76cd59e5ed6e08a75020f0d50fe69c4bebee3

##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/RunningCompaction.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.coordinator;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+
+public class RunningCompaction {
+
+  private final TExternalCompactionJob job;
+  private final String compactorAddress;
+  private final TServerInstance tserver;
+  private final Map<Long,CompactionUpdate> updates = new TreeMap<>();
+
+  RunningCompaction(TExternalCompactionJob job, String compactorAddress, TServerInstance tserver) {
+    super();
+    this.job = job;
+    this.compactorAddress = compactorAddress;
+    this.tserver = tserver;
+  }
+
+  public Map<Long,CompactionUpdate> getUpdates() {
+    return updates;
+  }
+
+  public void addUpdate(Long timestamp, String message, TCompactionState state) {
+    this.updates.put(timestamp, new CompactionUpdate(timestamp, message, state));
+  }
+
+  public TExternalCompactionJob getJob() {
+    return job;
+  }
+
+  public String getCompactorAddress() {
+    return compactorAddress;
+  }
+
+  public TServerInstance getTserver() {

Review comment:
       resolved in 90e76cd59e5ed6e08a75020f0d50fe69c4bebee3




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r637238906



##########
File path: server/compactor/src/main/java/org/apache/accumulo/compactor/Compactor.java
##########
@@ -0,0 +1,898 @@
+/*
+ * 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.compactor;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService.Client;
+import org.apache.accumulo.core.compaction.thrift.CompactorService;
+import org.apache.accumulo.core.compaction.thrift.CompactorService.Iface;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.iteratorsImpl.system.SystemIteratorUtil;
+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.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionKind;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.trace.TraceUtil;
+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.ServerServices;
+import org.apache.accumulo.core.util.ServerServices.Service;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.core.util.threads.Threads;
+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.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.ServerOpts;
+import org.apache.accumulo.server.compaction.CompactionInfo;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.hadoop.fs.Path;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.google.common.base.Preconditions;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class Compactor extends AbstractServer implements CompactorService.Iface {
+
+  public static class CompactorServerOpts extends ServerOpts {
+    @Parameter(required = true, names = {"-q", "--queue"}, description = "compaction queue name")
+    private String queueName = null;
+
+    public String getQueueName() {
+      return queueName;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(Compactor.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long TIME_BETWEEN_CANCEL_CHECKS = 5 * 60 * 1000;
+
+  private static final long TEN_MEGABYTES = 10485760;
+  private static final CompactionCoordinatorService.Client.Factory COORDINATOR_CLIENT_FACTORY =
+      new CompactionCoordinatorService.Client.Factory();
+
+  protected static final CompactionJobHolder JOB_HOLDER = new CompactionJobHolder();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  private final UUID compactorId = UUID.randomUUID();
+  private final AccumuloConfiguration aconf;
+  private final String queueName;
+  protected final AtomicReference<ExternalCompactionId> currentCompactionId =
+      new AtomicReference<>();
+
+  private SecurityOperation security;
+  private ServiceLock compactorLock;
+  private ServerAddress compactorAddress = null;
+
+  // Exposed for tests
+  protected volatile Boolean shutdown = false;
+
+  private final AtomicBoolean compactionRunning = new AtomicBoolean(false);
+
+  protected Compactor(CompactorServerOpts opts, String[] args) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = getConfiguration();
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected Compactor(CompactorServerOpts opts, String[] args, AccumuloConfiguration conf) {
+    super("compactor", opts, args);
+    queueName = opts.getQueueName();
+    aconf = conf;
+    setupSecurity();
+    var schedExecutor = ThreadPools.createGeneralScheduledExecutorService(aconf);
+    startGCLogger(schedExecutor);
+    startCancelChecker(schedExecutor, TIME_BETWEEN_CANCEL_CHECKS);
+    printStartupMsg();
+  }
+
+  protected void setupSecurity() {
+    getContext().setupCrypto();
+    security = AuditedSecurityOperation.getInstance(getContext());
+  }
+
+  protected void startGCLogger(ScheduledThreadPoolExecutor schedExecutor) {
+    schedExecutor.scheduleWithFixedDelay(() -> gcLogger.logGCInfo(getConfiguration()), 0,
+        TIME_BETWEEN_GC_CHECKS, TimeUnit.MILLISECONDS);
+  }
+
+  protected void startCancelChecker(ScheduledThreadPoolExecutor schedExecutor,
+      long timeBetweenChecks) {
+    schedExecutor.scheduleWithFixedDelay(() -> checkIfCanceled(), 0, timeBetweenChecks,
+        TimeUnit.MILLISECONDS);
+  }
+
+  protected void checkIfCanceled() {
+    TExternalCompactionJob job = JOB_HOLDER.getJob();
+    if (job != null) {
+      try {
+        var extent = KeyExtent.fromThrift(job.getExtent());
+        var ecid = ExternalCompactionId.of(job.getExternalCompactionId());
+
+        TabletMetadata tabletMeta =
+            getContext().getAmple().readTablet(extent, ColumnType.ECOMP, ColumnType.PREV_ROW);
+        if (tabletMeta == null || !tabletMeta.getExtent().equals(extent)
+            || !tabletMeta.getExternalCompactions().containsKey(ecid)) {
+          // table was deleted OR tablet was split or merged OR tablet no longer thinks compaction
+          // is running for some reason
+          LOG.info("Cancelling compaction {} that no longer has a metadata entry at {}", ecid,
+              extent);
+          JOB_HOLDER.cancel(job.getExternalCompactionId());
+          return;
+        }
+
+        if (job.getKind() == TCompactionKind.USER) {
+          String zTablePath = Constants.ZROOT + "/" + getContext().getInstanceID()
+              + Constants.ZTABLES + "/" + extent.tableId() + Constants.ZTABLE_COMPACT_CANCEL_ID;
+          byte[] id = getContext().getZooCache().get(zTablePath);
+          if (id == null) {
+            // table probably deleted
+            LOG.info("Cancelling compaction {} for table that no longer exists {}", ecid, extent);
+            JOB_HOLDER.cancel(job.getExternalCompactionId());
+            return;
+          } else {
+            var cancelId = Long.parseLong(new String(id, UTF_8));
+
+            if (cancelId >= job.getUserCompactionId()) {
+              LOG.info("Cancelling compaction {} because user compaction was canceled");
+              JOB_HOLDER.cancel(job.getExternalCompactionId());
+              return;
+            }
+          }
+        }
+      } catch (RuntimeException e) {
+        LOG.warn("Failed to check if compaction {} for {} was canceled.",
+            job.getExternalCompactionId(), KeyExtent.fromThrift(job.getExtent()), e);
+      }
+    }
+  }
+
+  protected void printStartupMsg() {
+    LOG.info("Version " + Constants.VERSION);
+    LOG.info("Instance " + getContext().getInstanceID());
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   *
+   * @param clientAddress
+   *          address of this Compactor
+   *
+   * @throws KeeperException
+   *           zookeeper error
+   * @throws InterruptedException
+   *           thread interrupted
+   */
+  protected void announceExistence(HostAndPort clientAddress)
+      throws KeeperException, InterruptedException {
+
+    String hostPort = ExternalCompactionUtil.getHostPortString(clientAddress);
+
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    String compactorQueuePath =
+        getContext().getZooKeeperRoot() + Constants.ZCOMPACTORS + "/" + this.queueName;
+    String zPath = compactorQueuePath + "/" + hostPort;
+
+    try {
+      zoo.mkdirs(compactorQueuePath);
+      zoo.putPersistentData(zPath, 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;
+    }
+
+    compactorLock = new ServiceLock(getContext().getZooReaderWriter().getZooKeeper(),
+        ServiceLock.path(zPath), compactorId);
+    LockWatcher lw = new LockWatcher() {
+      @Override
+      public void lostLock(final LockLossReason reason) {
+        Halt.halt(1, () -> {
+          LOG.error("Compactor lost lock (reason = {}), exiting.", reason);
+          gcLogger.logGCInfo(getConfiguration());
+        });
+      }
+
+      @Override
+      public void unableToMonitorLockNode(final Exception e) {
+        Halt.halt(1, () -> LOG.error("Lost ability to monitor Compactor lock, exiting.", e));
+      }
+    };
+
+    try {
+      byte[] lockContent =
+          new ServerServices(hostPort, Service.COMPACTOR_CLIENT).toString().getBytes(UTF_8);
+      for (int i = 0; i < 25; i++) {
+        zoo.putPersistentData(zPath, new byte[0], NodeExistsPolicy.SKIP);
+
+        if (compactorLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained Compactor lock {}", compactorLock.getLockPath());
+          return;
+        }
+        LOG.info("Waiting for Compactor 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 tablet server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Start this Compactors thrift service to handle incoming client requests
+   *
+   * @return address of this compactor client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startCompactorClientService() throws UnknownHostException {
+    Iface rpcProxy = TraceUtil.wrapService(this);
+    if (getContext().getThriftServerType() == ThriftServerType.SASL) {
+      rpcProxy = TCredentialsUpdatingWrapper.service(rpcProxy, getClass(), getConfiguration());
+    }
+    final CompactorService.Processor<Iface> processor = new CompactorService.Processor<>(rpcProxy);
+    Property maxMessageSizeProperty = (aconf.get(Property.COMPACTOR_MAX_MESSAGE_SIZE) != null
+        ? Property.COMPACTOR_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getMetricsSystem(), getContext(), getHostname(),
+        Property.COMPACTOR_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.COMPACTOR_PORTSEARCH, Property.COMPACTOR_MINTHREADS,
+        Property.COMPACTOR_MINTHREADS_TIMEOUT, Property.COMPACTOR_THREADCHECK,
+        maxMessageSizeProperty);
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  /**
+   * Called by a CompactionCoordinator to cancel the currently running compaction
+   *
+   * @param tinfo
+   *          trace info
+   * @param credentials
+   *          caller credentials
+   * @param externalCompactionId
+   *          compaction id
+   * @throws UnknownCompactionIdException
+   *           if the externalCompactionId does not match the currently executing compaction
+   */
+  @Override
+  public void cancel(TInfo tinfo, TCredentials credentials, String externalCompactionId)
+      throws TException {
+    // do not expect users to call this directly, expect other tservers to call this method
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+    cancel(externalCompactionId);
+  }
+
+  /**
+   * Cancel the compaction with this id.
+   *
+   * @param externalCompactionId
+   *          compaction id
+   * @throws UnknownCompactionIdException
+   *           if the externalCompactionId does not match the currently executing compaction
+   * @throws TException
+   *           thrift error
+   */
+  private void cancel(String externalCompactionId) throws TException {
+    if (JOB_HOLDER.cancel(externalCompactionId)) {
+      LOG.info("Cancel requested for compaction job {}", externalCompactionId);
+    } else {
+      throw new UnknownCompactionIdException();
+    }
+  }
+
+  /**
+   * Send an update to the CompactionCoordinator for this job
+   *
+   * @param job
+   *          compactionJob
+   * @param state
+   *          updated state
+   * @param message
+   *          updated message
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected void updateCompactionState(TExternalCompactionJob job, TCompactionState state,
+      String message) throws RetriesExceededException {
+    RetryableThriftCall<String> thriftCall = new RetryableThriftCall<>(1000,
+        RetryableThriftCall.MAX_WAIT_TIME, 25, new RetryableThriftFunction<String>() {
+          @Override
+          public String execute() throws TException {
+            Client coordinatorClient = getCoordinatorClient();
+            try {
+              coordinatorClient.updateCompactionStatus(TraceUtil.traceInfo(),
+                  getContext().rpcCreds(), job.getExternalCompactionId(), state, message,
+                  System.currentTimeMillis());
+              return "";
+            } finally {
+              ThriftUtil.returnClient(coordinatorClient);
+            }
+          }
+        });
+    thriftCall.run();
+  }
+
+  /**
+   * Notify the CompactionCoordinator the job failed
+   *
+   * @param job
+   *          current compaction job
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected void updateCompactionFailed(TExternalCompactionJob job)
+      throws RetriesExceededException {
+    RetryableThriftCall<String> thriftCall = new RetryableThriftCall<>(1000,
+        RetryableThriftCall.MAX_WAIT_TIME, 25, new RetryableThriftFunction<String>() {
+          @Override
+          public String execute() throws TException {
+            Client coordinatorClient = getCoordinatorClient();
+            try {
+              coordinatorClient.compactionFailed(TraceUtil.traceInfo(), getContext().rpcCreds(),
+                  job.getExternalCompactionId(), job.extent);
+              return "";
+            } finally {
+              ThriftUtil.returnClient(coordinatorClient);
+            }
+          }
+        });
+    thriftCall.run();
+  }
+
+  /**
+   * Update the CompactionCoordinator with the stats from the completed job
+   *
+   * @param job
+   *          current compaction job
+   * @param stats
+   *          compaction stats
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected void updateCompactionCompleted(TExternalCompactionJob job, TCompactionStats stats)
+      throws RetriesExceededException {
+    RetryableThriftCall<String> thriftCall = new RetryableThriftCall<>(1000,
+        RetryableThriftCall.MAX_WAIT_TIME, 25, new RetryableThriftFunction<String>() {
+          @Override
+          public String execute() throws TException {
+            Client coordinatorClient = getCoordinatorClient();
+            try {
+              coordinatorClient.compactionCompleted(TraceUtil.traceInfo(), getContext().rpcCreds(),
+                  job.getExternalCompactionId(), job.extent, stats);
+              return "";
+            } finally {
+              ThriftUtil.returnClient(coordinatorClient);
+            }
+          }
+        });
+    thriftCall.run();
+  }
+
+  /**
+   * Get the next job to run
+   *
+   * @param uuid
+   *          uuid supplier
+   * @return CompactionJob
+   * @throws RetriesExceededException
+   *           thrown when retries have been exceeded
+   */
+  protected TExternalCompactionJob getNextJob(Supplier<UUID> uuid) throws RetriesExceededException {
+    RetryableThriftCall<TExternalCompactionJob> nextJobThriftCall =
+        new RetryableThriftCall<>(1000, RetryableThriftCall.MAX_WAIT_TIME, 0,
+            new RetryableThriftFunction<TExternalCompactionJob>() {
+              @Override
+              public TExternalCompactionJob execute() throws TException {
+                Client coordinatorClient = getCoordinatorClient();
+                try {
+                  ExternalCompactionId eci = ExternalCompactionId.generate(uuid.get());
+                  LOG.trace("Attempting to get next job, eci = {}", eci);
+                  currentCompactionId.set(eci);
+                  return coordinatorClient.getCompactionJob(TraceUtil.traceInfo(),
+                      getContext().rpcCreds(), queueName,
+                      ExternalCompactionUtil.getHostPortString(compactorAddress.getAddress()),
+                      eci.toString());
+                } catch (Exception e) {
+                  currentCompactionId.set(null);
+                  throw e;
+                } finally {
+                  ThriftUtil.returnClient(coordinatorClient);
+                }
+              }
+            });
+    return nextJobThriftCall.run();
+  }
+
+  /**
+   * Get the client to the CompactionCoordinator
+   *
+   * @return compaction coordinator client
+   * @throws TTransportException
+   *           when unable to get client
+   */
+  protected CompactionCoordinatorService.Client getCoordinatorClient() throws TTransportException {
+    HostAndPort coordinatorHost = ExternalCompactionUtil.findCompactionCoordinator(getContext());
+    if (null == coordinatorHost) {
+      throw new TTransportException("Unable to get CompactionCoordinator address from ZooKeeper");
+    }
+    LOG.trace("CompactionCoordinator address is: {}", coordinatorHost);
+    return ThriftUtil.getClient(COORDINATOR_CLIENT_FACTORY, coordinatorHost, getContext());
+  }
+
+  /**
+   * Create compaction runnable
+   *
+   * @param job
+   *          compaction job
+   * @param totalInputEntries
+   *          object to capture total entries
+   * @param totalInputBytes
+   *          object to capture input file size
+   * @param started
+   *          started latch
+   * @param stopped
+   *          stopped latch
+   * @param err
+   *          reference to error
+   * @return Runnable compaction job
+   */
+  protected Runnable createCompactionJob(final TExternalCompactionJob job,
+      final LongAdder totalInputEntries, final LongAdder totalInputBytes,
+      final CountDownLatch started, final CountDownLatch stopped,
+      final AtomicReference<Throwable> err) {
+
+    return new Runnable() {
+      @Override
+      public void run() {
+        // Its only expected that a single compaction runs at a time. Multiple compactions running
+        // at a time could cause odd behavior like out of order and unexpected thrift calls to the
+        // coordinator. This is a sanity check to ensure the expectation is met. Should this check
+        // ever fail, it means there is a bug elsewhere.
+        Preconditions.checkState(compactionRunning.compareAndSet(false, true));
+        try {
+          LOG.info("Starting up compaction runnable for job: {}", job);
+          updateCompactionState(job, TCompactionState.STARTED, "Compaction started");
+
+          final AccumuloConfiguration tConfig;
+
+          if (!job.getTableCompactionProperties().isEmpty()) {
+            tConfig = new ConfigurationCopy(DefaultConfiguration.getInstance());

Review comment:
       Looking how this works here and in the tserver code, thinking this is brittle and error prone. What if a non table props is used, like a general prop?  That could cause silent failures in this code.  Think we need to rework this, but not sure what to do ATM.  The peer code in the tserver is in CompactableImpl.reserveExternalCompaction()




-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   I was referring to [this](https://github.blog/2019-07-01-mark-files-as-viewed/). I think the default in the File Filter dropdown is to remove them from your view.


-- 
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.

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



[GitHub] [accumulo] DomGarguilo commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635326117



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableUtils.java
##########
@@ -534,10 +540,10 @@ public static AccumuloConfiguration getCompactionConfig(CompactionKind kind, Tab
   }
 
   static StoredTabletFile compact(Tablet tablet, CompactionJob job, Set<StoredTabletFile> jobFiles,
-      Long compactionId, boolean propogateDeletes, CompactableImpl.CompactionHelper helper,
-      List<IteratorSetting> iters, CompactionCheck compactionCheck, RateLimiter readLimiter,
-      RateLimiter writeLimiter, CompactionStats stats)
-      throws IOException, CompactionCanceledException {
+      Long compactionId, Set<StoredTabletFile> selectedFiles, boolean propogateDeletes,

Review comment:
       Looks like there are lots of occurrences of this typo actually. propogate vs propagate 




-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r633903466



##########
File path: server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
##########
@@ -0,0 +1,766 @@
+/*
+ * 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.coordinator;
+
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.coordinator.QueueSummaries.PrioTserver;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.clientImpl.ThriftTransportPool;
+import org.apache.accumulo.core.clientImpl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface;
+import org.apache.accumulo.core.compaction.thrift.Compactor;
+import org.apache.accumulo.core.compaction.thrift.TCompactionState;
+import org.apache.accumulo.core.compaction.thrift.UnknownCompactionIdException;
+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.TKeyExtent;
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.ExternalCompactionId;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.LocationType;
+import org.apache.accumulo.core.rpc.ThriftUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionQueueSummary;
+import org.apache.accumulo.core.tabletserver.thrift.TCompactionStats;
+import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob;
+import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
+import org.apache.accumulo.core.trace.TraceUtil;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.compaction.ExternalCompactionUtil;
+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.ZooReaderWriter;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.compaction.RetryableThriftCall;
+import org.apache.accumulo.server.compaction.RetryableThriftCall.RetriesExceededException;
+import org.apache.accumulo.server.compaction.RetryableThriftFunction;
+import org.apache.accumulo.server.manager.LiveTServerSet;
+import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TCredentialsUpdatingWrapper;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftServerType;
+import org.apache.accumulo.server.security.AuditedSecurityOperation;
+import org.apache.accumulo.server.security.SecurityOperation;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CompactionCoordinator extends AbstractServer
+    implements org.apache.accumulo.core.compaction.thrift.CompactionCoordinator.Iface,
+    LiveTServerSet.Listener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionCoordinator.class);
+  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
+  private static final long FIFTEEN_MINUTES =
+      TimeUnit.MILLISECONDS.convert(Duration.of(15, TimeUnit.MINUTES.toChronoUnit()));
+
+  protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
+
+  /* Map of compactionId to RunningCompactions */
+  protected static final Map<ExternalCompactionId,RunningCompaction> RUNNING =

Review comment:
       That could work w/ some slight tweaks.  The compactionFailed method in the coordinator is kinda thrifty and only takes a single ecid.  Could have package private non thrifty compactionFailed method in the coordinator that takes a collections ecids and is called by the dead compaction detector and the thrifty compactionFailed method.




-- 
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.

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



[GitHub] [accumulo] Manno15 commented on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-847914876


   > Nice, did you verify the data after testing?
   
   I have after each test run. No issue with data loss. However, I just re-ran verify and received an interesting error. The job itself completed successfully but then ended up with a compactor error. Server logs below.
   
   <details>
   
   ```
   2021-05-25T10:13:22,937 [compactor.Compactor] ERROR: Unhandled error occurred in Compactor
   
   java.lang.RuntimeException: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /accumulo/3737bcb7-377b-4ad9-be42-efede2692e16/coordinators/lock
   	at org.apache.accumulo.core.util.compaction.ExternalCompactionUtil.findCompactionCoordinator(ExternalCompactionUtil.java:86)
   	at org.apache.accumulo.compactor.Compactor.getCoordinatorClient(Compactor.java:501)
   	at org.apache.accumulo.compactor.Compactor$5.execute(Compactor.java:473)
   	at org.apache.accumulo.compactor.Compactor$5.execute(Compactor.java:470)
   	at org.apache.accumulo.server.compaction.RetryableThriftCall.run(RetryableThriftCall.java:107)
   	at org.apache.accumulo.compactor.Compactor.getNextJob(Compactor.java:490)
   	at org.apache.accumulo.compactor.Compactor.run(Compactor.java:663)
   	at java.base/java.lang.Thread.run(Thread.java:829)
   Caused by: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /accumulo/3737bcb7-377b-4ad9-be42-efede2692e16/coordinators/lock
   	at org.apache.zookeeper.KeeperException.create(KeeperException.java:102)
   	at org.apache.zookeeper.KeeperException.create(KeeperException.java:54)
   	at org.apache.zookeeper.ZooKeeper.getChildren(ZooKeeper.java:2746)
   	at org.apache.accumulo.fate.zookeeper.ServiceLock.getLockData(ServiceLock.java:653)
   	at org.apache.accumulo.core.util.compaction.ExternalCompactionUtil.findCompactionCoordinator(ExternalCompactionUtil.java:79)
   	... 7 more
   ```
   
   ```
   ERROR: Unable to call shutdownNow
   java.lang.NoSuchFieldException: executorService_
           at java.lang.Class.getDeclaredField(Class.java:2411) ~[?:?]
           at org.apache.accumulo.server.rpc.TServerUtils.stopTServer(TServerUtils.java:717) ~[accumulo-server-base-2.1.0-SNAPSHOT.jar:2.1.0-SNAPSHOT]
           at org.apache.accumulo.compactor.Compactor.run(Compactor.java:793) ~[accumulo-compactor-2.1.0-SNAPSHOT.jar:2.1.0-SNAPSHOT]
           at java.lang.Thread.run(Thread.java:829) [?:?]
   2021-05-25T10:13:22,939 [compactor.Compactor] INFO : stop requested. exiting ...
   2021-05-25T10:13:30,118 [compactor.Compactor] ERROR: Compactor lost lock (reason = LOCK_DELETED), exiting.
   ```
   </details>
   
   Everything seems to be up in the monitor so I am not quite sure what happened there.


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
dlmarion commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r634294265



##########
File path: server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
##########
@@ -1654,6 +1658,79 @@ public void compact(TInfo tinfo, TCredentials credentials, String lock, String t
     return ret;
   }
 
+  @Override
+  public List<TCompactionQueueSummary> getCompactionQueueInfo(TInfo tinfo, TCredentials credentials)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    return server.getCompactionManager().getCompactionQueueSummaries();
+  }
+
+  @Override
+  public TExternalCompactionJob reserveCompactionJob(TInfo tinfo, TCredentials credentials,
+      String queueName, long priority, String compactor, String externalCompactionId)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    ExternalCompactionId eci = ExternalCompactionId.of(externalCompactionId);
+
+    var extCompaction = server.getCompactionManager().reserveExternalCompaction(queueName, priority,
+        compactor, eci);
+
+    if (extCompaction != null) {
+      return extCompaction.toThrift();
+    }
+
+    return new TExternalCompactionJob();
+  }
+
+  @Override
+  public void compactionJobFinished(TInfo tinfo, TCredentials credentials,
+      String externalCompactionId, TKeyExtent extent, long fileSize, long entries)
+      throws ThriftSecurityException, TException {
+
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    server.getCompactionManager().commitExternalCompaction(
+        ExternalCompactionId.of(externalCompactionId), KeyExtent.fromThrift(extent),
+        server.getOnlineTablets(), fileSize, entries);
+  }
+
+  @Override
+  public void compactionJobFailed(TInfo tinfo, TCredentials credentials,
+      String externalCompactionId, TKeyExtent extent) throws TException {
+    if (!security.canPerformSystemActions(credentials)) {
+      throw new AccumuloSecurityException(credentials.getPrincipal(),
+          SecurityErrorCode.PERMISSION_DENIED).asThriftException();
+    }
+
+    server.getCompactionManager().externalCompactionFailed(
+        ExternalCompactionId.of(externalCompactionId), KeyExtent.fromThrift(extent),
+        server.getOnlineTablets());
+  }
+
+  @Override
+  public boolean isRunningExternalCompaction(TInfo tinfo, TCredentials credentials,

Review comment:
       The code in the CompactionCoordinator that called this method was removed yesterday when I removed TServer from RunningCompaction. This was resolved in 937d7eeb8059380138ab384eacdfe3aa79b688fb. Removing this simple method from the code reduced the Thrift changes in this PR by about 100K LOC.
   
   j/k.




-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842584575


   Omitting a lot of detail, the following are the basics of what the current system does.  The first step uses a pluggable planner.
   
   1. Plan compaction :  Take current tablets files, current running compactions for the tablet, and emit set of compactions jobs.  A compaction job is set of files to compact, a  priority, and a destination queue.
    2. Queue Jobs : An attempt to queue the jobs emitted from the planner is made by doing the following.
       1. Cancel anything that was previously queued by the planner for the tablet.  If this fails, go back to planning step and try again because things probably changed during planning.
       2. Queue the new job from the planner on the desired priority queues
    3. When an internal or external compactor thread finishes a task, it takes the next highest priority job from the queue and actually run the compaction.
   
   I don' think interface `BiFunction<Set<File>, CompactionConfig, Future<File>>` would be sufficient to achieve this functionality.  It would not support the current functionality of priority queues and canceling queued work when things change (new files arrive, job start/finish during planning). In the current impl the priority queue for local compactions is very precise.  For external compaction there is essentially a global priority queue that is approximate (eventually consistent) that may not always start the highest priority job next, but usually will.
   
   


-- 
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.

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



[GitHub] [accumulo] Manno15 commented on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-844219494


   During one of my basic runs testing this code (two servers, no agitation) I ran into an error in the logs. I did not notice any other issue though and I am not sure if this even interrupted anything. Screen shot below:
   
   
   ![image](https://user-images.githubusercontent.com/29436247/118840731-aa326780-b895-11eb-8f4c-20b6ceeab69f.png)
   
   
   <details>
   
   ```
   java.lang.IllegalStateException
   	at com.google.common.base.Preconditions.checkState(Preconditions.java:494)
   	at org.apache.accumulo.tserver.tablet.CompactableImpl.commitExternalCompaction(CompactableImpl.java:1055)
   	at org.apache.accumulo.tserver.compactions.CompactionManager.commitExternalCompaction(CompactionManager.java:474)
   	at org.apache.accumulo.tserver.ThriftClientHandler.compactionJobFinished(ThriftClientHandler.java:1705)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.apache.accumulo.core.trace.TraceUtil.lambda$wrapService$6(TraceUtil.java:235)
   	at com.sun.proxy.$Proxy40.compactionJobFinished(Unknown Source)
   	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$compactionJobFinished.getResult(TabletClientService.java:4001)
   	at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$compactionJobFinished.getResult(TabletClientService.java:3982)
   	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:38)
   	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
   	at org.apache.accumulo.server.rpc.TimedProcessor.process(TimedProcessor.java:63)
   	at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:518)
   	at org.apache.accumulo.server.rpc.CustomNonBlockingServer$CustomFrameBuffer.invoke(CustomNonBlockingServer.java:114)
   	at org.apache.thrift.server.Invocation.run(Invocation.java:18)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:829)
   
   ```
   </details>
   


-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842584575


   Omitting a lot of detail, the following are the basics of what the current system does.  The first step uses a pluggable planner.
   
   1. Plan compaction :  Take current tablets files, current running compactions for the tablet, and emit set of compactions jobs.  A compaction job is set of files to compact, a  priority, and a destination queue.
    2. Queue Jobs : An attempt to queue the jobs emitted from the planner is made by doing the following.
       1. Cancel anything that was previously queued by the planner for the tablet.  If this fails, go back to planning step and try again because things probably changed during planning.  If the planner emits jobs that exactly match what was queued before, then skip the next step.
       2. Queue the new job from the planner on the desired priority queues
    3. When an internal or external compactor thread finishes a task, it takes the next highest priority job from the queue and actually runs a compaction.
   
   I don' think interface `BiFunction<Set<File>, CompactionConfig, Future<File>>` would be sufficient to achieve this functionality.  It would not support the current functionality of priority queues and canceling queued work when things change (new files arrive, job start/finish during planning). In the current impl the priority queue for local compactions is very precise.  For external compaction there is essentially a global priority queue that is approximate (eventually consistent) that may not always start the highest priority job next, but usually will.
   
   


-- 
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.

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



[GitHub] [accumulo] dlmarion commented on pull request #2096: External Compactions

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


   @Manno15  wrote:
   > This fills up the log file and it will eventually get pretty large with it mostly repeating the same information. Screenshot of the particular logs that I am talking about. I am not necessarily sure if this is an issue or not but is something I noticed.
   
   Yeah, we could probably turn that to debug. No issue there, just chatty


-- 
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.

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



[GitHub] [accumulo] keith-turner commented on a change in pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner commented on a change in pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#discussion_r635585325



##########
File path: core/src/main/java/org/apache/accumulo/core/spi/compaction/ExecutorManager.java
##########
@@ -31,4 +31,9 @@
    * Create a thread pool executor within a compaction service.
    */
   public CompactionExecutorId createExecutor(String name, int threads);
+
+  /**
+   * @return an id for a configured external execution queue.
+   */
+  public CompactionExecutorId getExternalExecutor(String name);

Review comment:
       The interface was not there so that users could implement their own, I was just using it to logically group things that went together to avoid having lots of unrelated groups of methods interleaved in a single interface.




-- 
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.

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



[GitHub] [accumulo] keith-turner edited a comment on pull request #2096: External Compactions

Posted by GitBox <gi...@apache.org>.
keith-turner edited a comment on pull request #2096:
URL: https://github.com/apache/accumulo/pull/2096#issuecomment-842460992


   > My understanding of the original idea to create an "external" compactions, was to create them "external" to Accumulo... something Accumulo could "submit" work to do and not care about the implementation, and get back a new compacted file.
   
   That was never an explicit goal I had. To achieve this goal, thinking about the following.  Currently the implementation of the coordinator and compactors components use a lot of internal Accumulo code.  To move these to external repo would require refactoring them to use only API and SPI calls into Accumulo. To do this would require exposing a lot internal low level API/SPI calls related to compactions(not sure what these would be, that would need exploration).   For this exercise to be useful these newly exposed API/SPI calls would have to be generally useful and not overfit (meaning the new API/SPI calls can only be used to implement the current design).  To avoid overfitting, it seems we would need to have multiple other designs in mind that we could test against these new API/SPI calls to verify that we have not overfit.  Is there another path other than what I have described?  Does this path actually lower the complexity? I am not sure it would, I think the answer to that de
 pends on what these new API/SPI calls are.
   


-- 
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.

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