diff --git a/assemble/pom.xml b/assemble/pom.xml
index 4b7fe169685..771b6f3c62e 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -283,6 +283,7 @@
true
+
org.apache.logging.log4j
log4j-1.2-api
true
diff --git a/core/pom.xml b/core/pom.xml
index fcaecd8c8c6..990f853a598 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -113,11 +113,6 @@
org.apache.hadoop
hadoop-client-api
-
-
- org.apache.logging.log4j
- log4j-1.2-api
-
org.apache.thrift
libthrift
@@ -142,26 +137,31 @@
org.yaml
snakeyaml
+
+ org.apache.hadoop
+ hadoop-client-runtime
+ runtime
+
biz.aQute.bnd
biz.aQute.bnd.annotation
- provided
+ test
-
- org.apache.logging.log4j
- log4j-api
- provided
+ org.apache.hadoop
+ hadoop-client-minicluster
+ test
- org.apache.hadoop
- hadoop-client-runtime
- runtime
+
+ org.apache.logging.log4j
+ log4j-1.2-api
+ test
- org.apache.hadoop
- hadoop-client-minicluster
+ org.apache.logging.log4j
+ log4j-api
test
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 dd02f6ba2f4..2ca9f69fb56 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 enum CompactionReason {
* @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 61671b85f46..cd6816435b7 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 33d307ce594..68d63cfac3d 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.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 e6fe10356cd..00000000000
--- 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 793f00cf47b..648504bcee9 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 {
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 32f0deac8fb..86cb8fbe231 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 324fafebc8c..6593237ab10 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 ThreadPoolExecutor createExecutorService(final AccumuloConfiguration conf
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 2723f3abe7f..b68ac376826 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 void testQuotes() {
@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 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 void testParseTreesOrdering() {
}
@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 6af9c87a576..9aaf67a8ef6 100644
--- a/minicluster/pom.xml
+++ b/minicluster/pom.xml
@@ -113,6 +113,13 @@
hadoop-client-runtime
runtime
+
+
+ org.apache.logging.log4j
+ log4j-1.2-api
+ runtime
+ true
+
org.apache.commons
commons-configuration2
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 2d5d83afa31..f4389f9fada 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 94c8fd380e1..3228cecaadf 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.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 DistributionStatisticConfig configure(Meter.Id id,
}
}
- // 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 f754b3936f5..f3bb320c3bc 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 void factoryTest() throws Exception {
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 2a8be3a157e..b48440b666e 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 @@ void zkNodeDeletedTest() throws Exception {
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 b5b63da26e9..ecf6709c623 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 caed9081d4b..639b0cb6f2a 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.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 SessionManager(ServerContext context) {
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 void run() {
}
};
- 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 73c2f6c614f..3d1bffedd28 100644
--- a/start/pom.xml
+++ b/start/pom.xml
@@ -44,12 +44,6 @@
spotbugs-annotations
test
-
-
- org.apache.logging.log4j
- log4j-1.2-api
- test
-
org.apache.logging.log4j
log4j-slf4j2-impl
diff --git a/test/pom.xml b/test/pom.xml
index 7db23647423..442256cadcf 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -162,10 +162,6 @@
-
- org.apache.logging.log4j
- log4j-1.2-api
-
org.apache.thrift
libthrift
@@ -215,6 +211,12 @@
hadoop-client-runtime
runtime
+
+
+ org.apache.logging.log4j
+ log4j-1.2-api
+ runtime
+
org.apache.accumulo
accumulo-iterator-test-harness
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 18fdff715d8..3e69c848975 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.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 FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes at
}
}
- /**
- * 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 getFileSizeMap(AccumuloClient client, String tableName) {
- var tableId = TableId.of(client.tableOperations().tableIdMap().get(tableName));
- Map 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()) {