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/12 19:15:52 UTC

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

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