This is an automated email from the ASF dual-hosted git repository.

ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new d54b4e57bc Cleanup of main branch (#4747)
d54b4e57bc is described below

commit d54b4e57bc0359228586c683c652136775dd8f3a
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Tue Jul 23 23:57:04 2024 -0400

    Cleanup of main branch (#4747)
    
    * Cleanup of main branch
    
    * Remove log4j-1.2-api dependency except as a bridge in the binary
      assembly (left a note in the assemble/pom.xml), and as a runtime
      dependency of hadoop minicluster, in the proper scope where needed
    * Move log4j dependencies in core back to test scope. These were moved
      to the provided scope in #4743 to work around an issue because
      log4j-1.2-api was still used in the core module in 2.1; now that the
      legacy log4j stuff was removed from core in 3.1 and later, we can move
      these back to the test scope where they belong as #4558 intended
    * Specify the exact version for deprecation "since" parameters (3.1.0,
      instead of 3.1)
    * Drop the unused and deprecated `tserver.workq.threads` property
    * Remove a few unused imports and variables, and drop the legacy
      MeterRegistryFactory that was deprecated in 2.1.3
    * Suppress some deprecation warnings and use var in a few places for
      readability
    * Delete unused test code in CompactionIT
---
 assemble/pom.xml                                   |  1 +
 core/pom.xml                                       | 30 ++++----
 .../core/client/admin/ActiveCompaction.java        |  2 +-
 .../org/apache/accumulo/core/conf/Property.java    | 42 +++++------
 .../core/metadata/schema/TabletMetadata.java       |  1 -
 .../core/metrics/MeterRegistryFactory.java         | 30 --------
 .../core/spi/balancer/data/TabletStatistics.java   |  2 +-
 .../core/spi/compaction/CompactionKind.java        |  5 +-
 .../accumulo/core/util/threads/ThreadPools.java    |  6 --
 .../core/security/ColumnVisibilityTest.java        | 83 ++++++++++++----------
 minicluster/pom.xml                                |  7 ++
 .../server/compaction/RetryableThriftCall.java     |  2 -
 .../accumulo/server/metrics/MetricsInfoImpl.java   | 24 ++-----
 .../server/metrics/MetricsInfoImplTest.java        | 17 -----
 .../accumulo/server/util/ServiceStatusCmdTest.java |  2 -
 .../coordinator/CompactionCoordinator.java         |  1 -
 .../accumulo/tserver/session/SessionManager.java   |  6 +-
 start/pom.xml                                      |  6 --
 test/pom.xml                                       | 10 +--
 .../accumulo/test/functional/CompactionIT.java     | 31 --------
 20 files changed, 102 insertions(+), 206 deletions(-)

diff --git a/assemble/pom.xml b/assemble/pom.xml
index 4b7fe16968..771b6f3c62 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -283,6 +283,7 @@
       <optional>true</optional>
     </dependency>
     <dependency>
+      <!-- we don't use this ourselves, but this bridges any dependencies 
using log4j1 to log4j2 -->
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-1.2-api</artifactId>
       <optional>true</optional>
diff --git a/core/pom.xml b/core/pom.xml
index fcaecd8c8c..990f853a59 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -113,11 +113,6 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client-api</artifactId>
     </dependency>
-    <dependency>
-      <!-- used at compile scope by legacy mapreduce classes -->
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-1.2-api</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.apache.thrift</groupId>
       <artifactId>libthrift</artifactId>
@@ -142,26 +137,31 @@
       <groupId>org.yaml</groupId>
       <artifactId>snakeyaml</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-runtime</artifactId>
+      <scope>runtime</scope>
+    </dependency>
     <dependency>
       <!-- used by log4j-api; needed here for compile time linting only; we 
don't actually use it -->
       <groupId>biz.aQute.bnd</groupId>
       <artifactId>biz.aQute.bnd.annotation</artifactId>
-      <scope>provided</scope>
+      <scope>test</scope>
     </dependency>
     <dependency>
-      <!-- used indirectly at compile scope by log4j-1.2-api; also needed for 
tests -->
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-api</artifactId>
-      <scope>provided</scope>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client-minicluster</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client-runtime</artifactId>
-      <scope>runtime</scope>
+      <!-- legacy log4j is still used by hadoop-client-minicluster -->
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-1.2-api</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client-minicluster</artifactId>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-api</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
 
b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
index dd02f6ba2f..2ca9f69fb5 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/client/admin/ActiveCompaction.java
@@ -57,7 +57,7 @@ public abstract class ActiveCompaction {
      * @deprecated Chop compactions no longer occur and it's not expected that 
listing compaction
      *             would ever return this.
      */
-    @Deprecated(since = "3.1")
+    @Deprecated(since = "3.1.0")
     CHOP,
     /**
      * idle compaction
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java 
b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 61671b85f4..cd6816435b 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -526,7 +526,7 @@ public enum Property {
       "The amount of memory used to store write-ahead-log mutations before 
flushing them.",
       "1.7.0"),
   @ReplacedBy(property = SPLIT_MAXOPEN)
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   
TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN("tserver.tablet.split.midpoint.files.max",
 "300",
       PropertyType.COUNT,
       "To find a tablets split points, all RFiles are opened and their indexes"
@@ -618,15 +618,15 @@ public enum Property {
       "The maximum number of concurrent tablet migrations for a tablet 
server.", "1.3.5"),
   TSERV_MAJC_DELAY("tserver.compaction.major.delay", "30s", 
PropertyType.TIMEDURATION,
       "Time a tablet server will sleep between checking which tablets need 
compaction.", "1.3.5"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   @ReplacedBy(property = COMPACTION_SERVICE_PREFIX)
   TSERV_COMPACTION_SERVICE_PREFIX("tserver.compaction.major.service.", null, 
PropertyType.PREFIX,
       "Prefix for compaction services.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   
TSERV_COMPACTION_SERVICE_ROOT_PLANNER("tserver.compaction.major.service.root.planner",
       DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
       "Compaction planner for root tablet service.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   
TSERV_COMPACTION_SERVICE_ROOT_RATE_LIMIT("tserver.compaction.major.service.root.rate.limit",
 "0B",
       PropertyType.BYTES,
       "Maximum number of bytes to read or write per second over all major"
@@ -634,11 +634,11 @@ public enum Property {
           + " been deprecated in anticipation of it being removed in a future 
release that"
           + " removes the rate limiting feature.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_ROOT_MAX_OPEN(
       "tserver.compaction.major.service.root.planner.opts.maxOpen", "30", 
PropertyType.COUNT,
       "The maximum number of files a compaction will open.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_ROOT_EXECUTORS(
       "tserver.compaction.major.service.root.planner.opts.executors",
       
"[{'name':'small','type':'internal','maxSize':'32M','numThreads':1},{'name':'huge','type':'internal','numThreads':1}]"
@@ -646,11 +646,11 @@ public enum Property {
       PropertyType.STRING,
       "See {% jlink -f 
org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   
TSERV_COMPACTION_SERVICE_META_PLANNER("tserver.compaction.major.service.meta.planner",
       DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
       "Compaction planner for metadata table.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   
TSERV_COMPACTION_SERVICE_META_RATE_LIMIT("tserver.compaction.major.service.meta.rate.limit",
 "0B",
       PropertyType.BYTES,
       "Maximum number of bytes to read or write per second over all major"
@@ -658,11 +658,11 @@ public enum Property {
           + " been deprecated in anticipation of it being removed in a future 
release that"
           + " removes the rate limiting feature.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_META_MAX_OPEN(
       "tserver.compaction.major.service.meta.planner.opts.maxOpen", "30", 
PropertyType.COUNT,
       "The maximum number of files a compaction will open.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_META_EXECUTORS(
       "tserver.compaction.major.service.meta.planner.opts.executors",
       
"[{'name':'small','type':'internal','maxSize':'32M','numThreads':2},{'name':'huge','type':'internal','numThreads':2}]"
@@ -670,12 +670,12 @@ public enum Property {
       PropertyType.JSON,
       "See {% jlink -f 
org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner %}.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_DEFAULT_PLANNER(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME + 
".planner",
       DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME,
       "Planner for default compaction service.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME + 
".rate.limit", "0B",
       PropertyType.BYTES,
@@ -684,12 +684,12 @@ public enum Property {
           + " been deprecated in anticipation of it being removed in a future 
release that"
           + " removes the rate limiting feature.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME
           + ".planner.opts.maxOpen",
       "10", PropertyType.COUNT, "The maximum number of files a compaction will 
open.", "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_COMPACTION_SERVICE_DEFAULT_EXECUTORS(
       "tserver.compaction.major.service." + DEFAULT_COMPACTION_SERVICE_NAME
           + ".planner.opts.executors",
@@ -700,7 +700,7 @@ public enum Property {
       "2.1.0"),
   TSERV_MINC_MAXCONCURRENT("tserver.compaction.minor.concurrent.max", "4", 
PropertyType.COUNT,
       "The maximum number of concurrent minor compactions for a tablet 
server.", "1.3.5"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   @ReplacedBy(property = COMPACTION_WARN_TIME)
   TSERV_COMPACTION_WARN_TIME("tserver.compaction.warn.time", "10m", 
PropertyType.TIMEDURATION,
       "When a compaction has not made progress for this time period, a warning 
will be logged.",
@@ -758,12 +758,6 @@ public enum Property {
           + " that begin with 'table.file' can be used here. For example, to 
set the compression"
           + " of the sorted recovery files to snappy use 
'tserver.wal.sort.file.compress.type=snappy'.",
       "2.1.0"),
-  @Deprecated(since = "2.1.3")
-  TSERV_WORKQ_THREADS("tserver.workq.threads", "2", PropertyType.COUNT,
-      "The number of threads for the distributed work queue. These threads are"
-          + " used for copying failed bulk import RFiles. Note that as of 
version 3.1.0 this property"
-          + " is not used and will be removed in a future release.",
-      "1.4.2"),
   TSERV_WAL_SYNC("tserver.wal.sync", "true", PropertyType.BOOLEAN,
       "Use the SYNC_BLOCK create flag to sync WAL writes to disk. Prevents"
           + " problems recovering from sudden system resets.",
@@ -801,7 +795,7 @@ public enum Property {
       "The number of threads on each tablet server available to retrieve"
           + " summary data, that is not currently in cache, from RFiles.",
       "2.0.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TSERV_LAST_LOCATION_MODE("tserver.last.location.mode", "compaction",
       PropertyType.LAST_LOCATION_MODE,
       "Describes how the system will record the 'last' location for tablets, 
which can be used for"
@@ -936,13 +930,13 @@ public enum Property {
           + "specified time.  If a system compaction cancels a hold and runs, 
then the user compaction"
           + " can reselect and hold files after the system compaction runs.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TABLE_COMPACTION_SELECTOR("table.compaction.selector", "", 
PropertyType.CLASSNAME,
       "A configurable selector for a table that can periodically select file 
for mandatory "
           + "compaction, even if the files do not meet the compaction ratio. 
This option was deprecated in "
           + "3.1, see the CompactionKind.SELECTOR enum javadoc for details.",
       "2.1.0"),
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   TABLE_COMPACTION_SELECTOR_OPTS("table.compaction.selector.opts.", null, 
PropertyType.PREFIX,
       "Options for the table compaction dispatcher.", "2.1.0"),
   TABLE_COMPACTION_CONFIGURER("table.compaction.configurer", "", 
PropertyType.CLASSNAME,
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
index 33d307ce59..68d63cfac3 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java
@@ -77,7 +77,6 @@ import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSortedMap.Builder;
 import com.google.common.net.HostAndPort;
 
 public class TabletMetadata {
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metrics/MeterRegistryFactory.java 
b/core/src/main/java/org/apache/accumulo/core/metrics/MeterRegistryFactory.java
deleted file mode 100644
index e6fe10356c..0000000000
--- 
a/core/src/main/java/org/apache/accumulo/core/metrics/MeterRegistryFactory.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.accumulo.core.metrics;
-
-import io.micrometer.core.instrument.MeterRegistry;
-
-/**
- * @deprecated since 2.1.3; use {@link 
org.apache.accumulo.core.spi.metrics.MeterRegistryFactory}
- *             instead
- */
-@Deprecated()
-public interface MeterRegistryFactory {
-  MeterRegistry create();
-}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
 
b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
index 793f00cf47..648504bcee 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/balancer/data/TabletStatistics.java
@@ -28,7 +28,7 @@ public interface TabletStatistics extends 
Comparable<TabletStatistics> {
 
   long getNumEntries();
 
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   long getSplitCreationTime();
 
   double getIngestRate();
diff --git 
a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
 
b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
index 32f0deac8f..86cb8fbe23 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/spi/compaction/CompactionKind.java
@@ -33,15 +33,14 @@ public enum CompactionKind {
   /**
    * Set of files selected by a {@link CompactionSelector} configured for a 
table.
    *
-   * @deprecated since 3.1. Use of selector compactions should be replaced 
with user compactions
-   *             initiated via
+   * @deprecated Use of selector compactions should be replaced with user 
compactions initiated via
    *             {@link 
org.apache.accumulo.core.client.admin.TableOperations#compact(String, 
CompactionConfig)}.
    *             Everything that can be done with selector compactions can 
also be done with user
    *             compactions. User compactions offer more control over when 
compactions run, the
    *             range of data compacted, and the ability to cancel. Selector 
compactions offer none
    *             of these features and were deprecated in favor of only 
offering user compactions.
    */
-  @Deprecated(since = "3.1")
+  @Deprecated(since = "3.1.0")
   SELECTOR,
   /**
    * A user initiated a one time compaction using an Accumulo client.
diff --git 
a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java 
b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
index 324fafebc8..6593237ab1 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
@@ -289,12 +289,6 @@ public class ThreadPools {
           builder.enableThreadPoolMetrics();
         }
         return builder.build();
-      case TSERV_WORKQ_THREADS:
-        builder = getPoolBuilder("distributed work 
queue").numCoreThreads(conf.getCount(p));
-        if (emitThreadPoolMetrics) {
-          builder.enableThreadPoolMetrics();
-        }
-        return builder.build();
       case TSERV_MINC_MAXCONCURRENT:
         builder = getPoolBuilder("minor 
compactor").numCoreThreads(conf.getCount(p)).withTimeOut(0L,
             MILLISECONDS);
diff --git 
a/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
 
b/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
index 2723f3abe7..b68ac37682 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
@@ -19,22 +19,26 @@
 package org.apache.accumulo.core.security;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.accumulo.core.security.ColumnVisibility.quote;
 import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.util.Comparator;
-
-import org.apache.accumulo.core.security.ColumnVisibility.Node;
-import org.apache.accumulo.core.security.ColumnVisibility.NodeComparator;
-import org.apache.accumulo.core.security.ColumnVisibility.NodeType;
 import org.apache.hadoop.io.Text;
 import org.junit.jupiter.api.Test;
 
 public class ColumnVisibilityTest {
 
+  @SuppressWarnings("deprecation")
+  private static org.apache.accumulo.core.security.ColumnVisibility.NodeType 
AND =
+      org.apache.accumulo.core.security.ColumnVisibility.NodeType.AND;
+  @SuppressWarnings("deprecation")
+  private static org.apache.accumulo.core.security.ColumnVisibility.NodeType 
OR =
+      org.apache.accumulo.core.security.ColumnVisibility.NodeType.OR;
+  @SuppressWarnings("deprecation")
+  private static org.apache.accumulo.core.security.ColumnVisibility.NodeType 
TERM =
+      org.apache.accumulo.core.security.ColumnVisibility.NodeType.TERM;
+
   private void shouldThrow(String... strings) {
     for (String s : strings) {
       final byte[] sBytes = s.getBytes();
@@ -165,75 +169,75 @@ public class ColumnVisibilityTest {
   @Test
   @SuppressWarnings("deprecation")
   public void testToString() {
-    ColumnVisibility cv = new ColumnVisibility(quote("a"));
+    ColumnVisibility cv = new ColumnVisibility(ColumnVisibility.quote("a"));
     assertEquals("[a]", cv.toString());
 
     // multi-byte
-    cv = new ColumnVisibility(quote("五"));
+    cv = new ColumnVisibility(ColumnVisibility.quote("五"));
     assertEquals("[\"五\"]", cv.toString());
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testParseTree() {
-    Node node = parse("(W)|(U&V)");
-    assertNode(node, NodeType.OR, 0, 9);
-    assertNode(node.getChildren().get(0), NodeType.TERM, 1, 2);
-    assertNode(node.getChildren().get(1), NodeType.AND, 5, 8);
+    var node = parse("(W)|(U&V)");
+    assertNode(node, OR, 0, 9);
+    assertNode(node.getChildren().get(0), TERM, 1, 2);
+    assertNode(node.getChildren().get(1), AND, 5, 8);
   }
 
   @Test
-  @SuppressWarnings("deprecation")
   public void testParseTreeWithNoChildren() {
-    Node node = parse("ABC");
-    assertNode(node, NodeType.TERM, 0, 3);
+    var node = parse("ABC");
+    assertNode(node, TERM, 0, 3);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testParseTreeWithTwoChildren() {
-    Node node = parse("ABC|DEF");
-    assertNode(node, NodeType.OR, 0, 7);
-    assertNode(node.getChildren().get(0), NodeType.TERM, 0, 3);
-    assertNode(node.getChildren().get(1), NodeType.TERM, 4, 7);
+    var node = parse("ABC|DEF");
+    assertNode(node, OR, 0, 7);
+    assertNode(node.getChildren().get(0), TERM, 0, 3);
+    assertNode(node.getChildren().get(1), TERM, 4, 7);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testParseTreeWithParenthesesAndTwoChildren() {
-    Node node = parse("(ABC|DEF)");
-    assertNode(node, NodeType.OR, 1, 8);
-    assertNode(node.getChildren().get(0), NodeType.TERM, 1, 4);
-    assertNode(node.getChildren().get(1), NodeType.TERM, 5, 8);
+    var node = parse("(ABC|DEF)");
+    assertNode(node, OR, 1, 8);
+    assertNode(node.getChildren().get(0), TERM, 1, 4);
+    assertNode(node.getChildren().get(1), TERM, 5, 8);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testParseTreeWithParenthesizedChildren() {
-    Node node = parse("ABC|(DEF&GHI)");
-    assertNode(node, NodeType.OR, 0, 13);
-    assertNode(node.getChildren().get(0), NodeType.TERM, 0, 3);
-    assertNode(node.getChildren().get(1), NodeType.AND, 5, 12);
-    assertNode(node.getChildren().get(1).children.get(0), NodeType.TERM, 5, 8);
-    assertNode(node.getChildren().get(1).children.get(1), NodeType.TERM, 9, 
12);
+    var node = parse("ABC|(DEF&GHI)");
+    assertNode(node, OR, 0, 13);
+    assertNode(node.getChildren().get(0), TERM, 0, 3);
+    assertNode(node.getChildren().get(1), AND, 5, 12);
+    assertNode(node.getChildren().get(1).children.get(0), TERM, 5, 8);
+    assertNode(node.getChildren().get(1).children.get(1), TERM, 9, 12);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testParseTreeWithMoreParentheses() {
-    Node node = parse("(W)|(U&V)");
-    assertNode(node, NodeType.OR, 0, 9);
-    assertNode(node.getChildren().get(0), NodeType.TERM, 1, 2);
-    assertNode(node.getChildren().get(1), NodeType.AND, 5, 8);
-    assertNode(node.getChildren().get(1).children.get(0), NodeType.TERM, 5, 6);
-    assertNode(node.getChildren().get(1).children.get(1), NodeType.TERM, 7, 8);
+    var node = parse("(W)|(U&V)");
+    assertNode(node, OR, 0, 9);
+    assertNode(node.getChildren().get(0), TERM, 1, 2);
+    assertNode(node.getChildren().get(1), AND, 5, 8);
+    assertNode(node.getChildren().get(1).children.get(0), TERM, 5, 6);
+    assertNode(node.getChildren().get(1).children.get(1), TERM, 7, 8);
   }
 
   @Test
   @SuppressWarnings("deprecation")
   public void testEmptyParseTreesAreEqual() {
-    Comparator<Node> comparator = new NodeComparator(new byte[] {});
-    Node empty = new ColumnVisibility().getParseTree();
+    var comparator =
+        new 
org.apache.accumulo.core.security.ColumnVisibility.NodeComparator(new byte[] 
{});
+    var empty = new ColumnVisibility().getParseTree();
     assertEquals(0, comparator.compare(empty, parse("")));
   }
 
@@ -250,13 +254,14 @@ public class ColumnVisibilityTest {
   }
 
   @SuppressWarnings("deprecation")
-  private Node parse(String s) {
+  private org.apache.accumulo.core.security.ColumnVisibility.Node parse(String 
s) {
     ColumnVisibility v = new ColumnVisibility(s);
     return v.getParseTree();
   }
 
   @SuppressWarnings("deprecation")
-  private void assertNode(Node node, NodeType nodeType, int start, int end) {
+  private void 
assertNode(org.apache.accumulo.core.security.ColumnVisibility.Node node,
+      org.apache.accumulo.core.security.ColumnVisibility.NodeType nodeType, 
int start, int end) {
     assertEquals(node.type, nodeType);
     assertEquals(start, node.start);
     assertEquals(end, node.end);
diff --git a/minicluster/pom.xml b/minicluster/pom.xml
index 6af9c87a57..9aaf67a8ef 100644
--- a/minicluster/pom.xml
+++ b/minicluster/pom.xml
@@ -113,6 +113,13 @@
       <artifactId>hadoop-client-runtime</artifactId>
       <scope>runtime</scope>
     </dependency>
+    <dependency>
+      <!-- legacy log4j is still used by hadoop-client-minicluster -->
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-1.2-api</artifactId>
+      <scope>runtime</scope>
+      <optional>true</optional>
+    </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-configuration2</artifactId>
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
 
b/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
index 2d5d83afa3..f4389f9fad 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/compaction/RetryableThriftCall.java
@@ -18,8 +18,6 @@
  */
 package org.apache.accumulo.server.compaction;
 
-import static java.time.Duration.ofMillis;
-
 import java.time.Duration;
 
 import org.apache.accumulo.core.util.Retry;
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
 
b/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
index 94c8fd380e..3228cecaad 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/metrics/MetricsInfoImpl.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.classloader.ClassLoaderUtil;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.metrics.MetricsInfo;
 import org.apache.accumulo.core.metrics.MetricsProducer;
+import org.apache.accumulo.core.spi.metrics.MeterRegistryFactory;
 import org.apache.accumulo.server.ServerContext;
 import org.checkerframework.checker.nullness.qual.NonNull;
 import org.slf4j.Logger;
@@ -266,35 +267,20 @@ public class MetricsInfoImpl implements MetricsInfo {
     }
   }
 
-  // support for org.apache.accumulo.core.metrics.MeterRegistryFactory can be 
removed in 3.1
   @VisibleForTesting
-  @SuppressWarnings("deprecation")
   static MeterRegistry getRegistryFromFactory(final String factoryName, final 
ServerContext context)
       throws ClassNotFoundException, NoSuchMethodException, 
InvocationTargetException,
       InstantiationException, IllegalAccessException {
     try {
       LOG.info("look for meter spi registry factory {}", factoryName);
-      Class<? extends 
org.apache.accumulo.core.spi.metrics.MeterRegistryFactory> clazz =
-          ClassLoaderUtil.loadClass(factoryName,
-              org.apache.accumulo.core.spi.metrics.MeterRegistryFactory.class);
-      org.apache.accumulo.core.spi.metrics.MeterRegistryFactory factory =
-          clazz.getDeclaredConstructor().newInstance();
-      org.apache.accumulo.core.spi.metrics.MeterRegistryFactory.InitParameters 
initParameters =
-          new MeterRegistryEnvPropImpl(context);
+      Class<? extends MeterRegistryFactory> clazz =
+          ClassLoaderUtil.loadClass(factoryName, MeterRegistryFactory.class);
+      MeterRegistryFactory factory = 
clazz.getDeclaredConstructor().newInstance();
+      MeterRegistryFactory.InitParameters initParameters = new 
MeterRegistryEnvPropImpl(context);
       return factory.create(initParameters);
     } catch (ClassCastException ex) {
       // empty. On exception try deprecated version
     }
-    try {
-      LOG.info("find legacy meter registry factory {}", factoryName);
-      Class<? extends org.apache.accumulo.core.metrics.MeterRegistryFactory> 
clazz = ClassLoaderUtil
-          .loadClass(factoryName, 
org.apache.accumulo.core.metrics.MeterRegistryFactory.class);
-      org.apache.accumulo.core.metrics.MeterRegistryFactory factory =
-          clazz.getDeclaredConstructor().newInstance();
-      return factory.create();
-    } catch (ClassCastException ex) {
-      // empty. No valid metrics factory, fall through and then throw 
exception.
-    }
     throw new ClassNotFoundException(
         "Could not find appropriate class implementing a MetricsFactory for: " 
+ factoryName);
   }
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/metrics/MetricsInfoImplTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/metrics/MetricsInfoImplTest.java
index f754b3936f..f3bb320c3b 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/metrics/MetricsInfoImplTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/metrics/MetricsInfoImplTest.java
@@ -47,29 +47,12 @@ public class MetricsInfoImplTest {
     replay(context, conf);
     
assertNotNull(MetricsInfoImpl.getRegistryFromFactory(SPIFactory.class.getName(),
 context));
 
-    assertNotNull(
-        
MetricsInfoImpl.getRegistryFromFactory(DeprecatedFactory.class.getName(), 
context));
-
     assertThrows(ClassNotFoundException.class,
         () -> MetricsInfoImpl.getRegistryFromFactory(String.class.getName(), 
context));
 
     verify(context, conf);
   }
 
-  // support for org.apache.accumulo.core.metrics.MeterRegistryFactory can be 
removed in 3.1
-  @SuppressWarnings("deprecation")
-  static final class DeprecatedFactory
-      implements org.apache.accumulo.core.metrics.MeterRegistryFactory {
-    DeprecatedFactory() {
-
-    }
-
-    @Override
-    public MeterRegistry create() {
-      return new SimpleMeterRegistry();
-    }
-  }
-
   static class SPIFactory implements 
org.apache.accumulo.core.spi.metrics.MeterRegistryFactory {
 
     SPIFactory() {
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
index 2a8be3a157..b48440b666 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/util/ServiceStatusCmdTest.java
@@ -452,8 +452,6 @@ public class ServiceStatusCmdTest {
     String host2 = "localhost:9992";
     String host3 = "hostA:9999";
 
-    String lock1data =
-        
"{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\"localhost:9991\",\"group\":\"default\"}]}";
     String lock2Data =
         
"{\"descriptors\":[{\"uuid\":\"6effb690-c29c-4e0b-92ff-f6b308385a42\",\"service\":\"MANAGER\",\"address\":\""
             + host2 + "\",\"group\":\"default\"}]}";
diff --git 
a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
 
b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
index b5b63da26e..ecf6709c62 100644
--- 
a/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
+++ 
b/server/compaction-coordinator/src/main/java/org/apache/accumulo/coordinator/CompactionCoordinator.java
@@ -109,7 +109,6 @@ public class CompactionCoordinator extends AbstractServer
 
   private static final Logger STATUS_LOG =
       LoggerFactory.getLogger(CompactionCoordinator.class.getName() + 
".compaction.status");
-  private static final long TIME_BETWEEN_GC_CHECKS = 5000;
   protected static final QueueSummaries QUEUE_SUMMARIES = new QueueSummaries();
 
   /*
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
index caed9081d4..639b0cb6f2 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/SessionManager.java
@@ -36,7 +36,6 @@ import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -80,7 +79,7 @@ public class SessionManager {
     Runnable r = () -> sweep(maxIdle, maxUpdateIdle);
 
     
ThreadPools.watchCriticalScheduledTask(context.getScheduledExecutor().scheduleWithFixedDelay(r,
-        0, Math.max(maxIdle / 2, 1000), TimeUnit.MILLISECONDS));
+        0, Math.max(maxIdle / 2, 1000), MILLISECONDS));
   }
 
   public long createSession(Session session, boolean reserve) {
@@ -332,8 +331,7 @@ public class SessionManager {
         }
       };
 
-      ScheduledFuture<?> future =
-          ctx.getScheduledExecutor().schedule(r, delay, TimeUnit.MILLISECONDS);
+      ScheduledFuture<?> future = ctx.getScheduledExecutor().schedule(r, 
delay, MILLISECONDS);
       ThreadPools.watchNonCriticalScheduledTask(future);
     }
   }
diff --git a/start/pom.xml b/start/pom.xml
index 73c2f6c614..3d1bffedd2 100644
--- a/start/pom.xml
+++ b/start/pom.xml
@@ -44,12 +44,6 @@
       <artifactId>spotbugs-annotations</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <!-- required to exclude log4j classes in some powermock tests -->
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-1.2-api</artifactId>
-      <scope>test</scope>
-    </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-slf4j2-impl</artifactId>
diff --git a/test/pom.xml b/test/pom.xml
index 7db2364742..442256cadc 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -162,10 +162,6 @@
         </exclusion>
       </exclusions>
     </dependency>
-    <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-1.2-api</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.apache.thrift</groupId>
       <artifactId>libthrift</artifactId>
@@ -215,6 +211,12 @@
       <artifactId>hadoop-client-runtime</artifactId>
       <scope>runtime</scope>
     </dependency>
+    <dependency>
+      <!-- legacy log4j is still used by hadoop-client-minicluster -->
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-1.2-api</artifactId>
+      <scope>runtime</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-iterator-test-harness</artifactId>
diff --git 
a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java 
b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
index 18fdff715d..3e69c84897 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java
@@ -28,7 +28,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
 import java.io.IOException;
-import java.io.UncheckedIOException;
 import java.nio.file.FileVisitResult;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -870,36 +869,6 @@ public class CompactionIT extends AccumuloClusterHarness {
     }
   }
 
-  /**
-   * Was used in debugging {@link #testGetSelectedFilesForCompaction}. May be 
useful later.
-   *
-   * @param client An accumulo client
-   * @param tableName The name of the table
-   * @return a map of the RFiles to their size in bytes
-   */
-  private Map<String,Long> getFileSizeMap(AccumuloClient client, String 
tableName) {
-    var tableId = 
TableId.of(client.tableOperations().tableIdMap().get(tableName));
-    Map<String,Long> map = new HashMap<>();
-
-    try (var tabletsMeta =
-        
TabletsMetadata.builder(client).forTable(tableId).fetch(ColumnType.FILES).build())
 {
-      for (TabletMetadata tm : tabletsMeta) {
-        for (StoredTabletFile stf : tm.getFiles()) {
-          try {
-            String filePath = stf.getPath().toString();
-            Long fileSize =
-                FileSystem.getLocal(new 
Configuration()).getFileStatus(stf.getPath()).getLen();
-            map.put(filePath, fileSize);
-          } catch (IOException e) {
-            throw new UncheckedIOException(e);
-          }
-        }
-      }
-
-      return map;
-    }
-  }
-
   @Test
   public void testDeleteCompactionService() throws Exception {
     try (AccumuloClient c = 
Accumulo.newClient().from(getClientProps()).build()) {


Reply via email to