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 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 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 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()) {