Skip to content

Commit

Permalink
Clean up the 2.1 build a bit (#4743)
Browse files Browse the repository at this point in the history
* Bump commons-logging to 1.3, so it natively supports sending logs to
  slf4j or log4j without the log4j-jcl bridge
  See apache/commons-logging#177
* Fix issue described in
  #4558 (comment)
  regarding log4j-api being required for log4j-1.2-api in core/pom.xml
  by making sure bnd and log4j-api are in the compile scope but marked
  as provided to avoid the maven-dependency-plugin from complaining
  because they appear to be required only for the test, but actually are
  still needed transitively for log4j-1.2-api
* Fix some false-positive Closeable resource warnings from wrapping a
  Scanner with an IsolatedScanner
* Suppress some deprecation warnings for scan server metadata schema
* Remove log4j-1.2-api from minicluster and compactor modules, where
  they weren't used
* Fix some incorrect `@SuppressWarnings` comments
* Fix generics with the ServiceStatusCmd's Result class, which had a
  generic `A extends Integer` parameter, but `Integer` is final and
  cannot be extended
* Remove some unused variables
* Add a few comments in the POMs to explain why some log4j dependencies
  exist where they do
* Add a try-with-resources block in ClientSideIteratorIT to ensure the
  scanner is closed when done
* Remove an unneeded exclusion on httpclient from libthrift (it's not a
  transitive dependency in the POM for the version of libthrift we are
  using, and upstream they have already migrated to newer versions of
  httpclient5 in the latest POM for libthrift)
  • Loading branch information
ctubbsii authored Jul 18, 2024
1 parent 0e7417d commit a7167da
Show file tree
Hide file tree
Showing 21 changed files with 56 additions and 81 deletions.
5 changes: 0 additions & 5 deletions assemble/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -302,11 +302,6 @@
<artifactId>log4j-core</artifactId>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-jcl</artifactId>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-jul</artifactId>
Expand Down
30 changes: 13 additions & 17 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -114,18 +114,13 @@
<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>
<exclusions>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>
Expand All @@ -148,24 +143,25 @@
<artifactId>snakeyaml</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-runtime</artifactId>
<scope>runtime</scope>
</dependency>
<!-- bnd dependency added due to lint issue, see https://github.com/apache/logging-log4j2/issues/2232 -->
<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>test</scope>
<scope>provided</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>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-minicluster</artifactId>
<scope>test</scope>
<artifactId>hadoop-client-runtime</artifactId>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-minicluster</artifactId>
<scope>test</scope>
</dependency>
<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -502,7 +502,9 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
}
if (isIsolated) {
log.info("Creating isolated scanner");
scanner = new IsolatedScanner(scanner);
@SuppressWarnings("resource")
var wrapped = new IsolatedScanner(scanner);
scanner = wrapped;
}
if (usesLocalIterators) {
log.info("Using local iterators");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -517,7 +517,9 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
}
if (isIsolated) {
log.info("Creating isolated scanner");
scanner = new IsolatedScanner(scanner);
@SuppressWarnings("resource")
var wrapped = new IsolatedScanner(scanner);
scanner = wrapped;
}
if (usesLocalIterators) {
log.info("Using local iterators");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.metadata.schema.MetadataSchema;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
import org.apache.accumulo.core.util.UuidUtil;
import org.apache.hadoop.fs.Path;
Expand All @@ -38,7 +37,9 @@
public class ScanServerRefTabletFile extends TabletFile {

@SuppressWarnings("deprecation")
private static final String OLD_PREFIX = OldScanServerFileReferenceSection.getRowPrefix();
private static final String OLD_PREFIX =
org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection
.getRowPrefix();
private final String prefix;
private final Value NULL_VALUE = new Value(new byte[0]);
private final Text serverAddress;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,9 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
}
if (isIsolated) {
log.info("Creating isolated scanner");
scanner = new IsolatedScanner(scanner);
@SuppressWarnings("resource")
var wrapped = new IsolatedScanner(scanner);
scanner = wrapped;
}
if (usesLocalIterators) {
log.info("Using local iterators");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,7 +214,9 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
}
if (isIsolated) {
log.info("Creating isolated scanner");
scanner = new IsolatedScanner(scanner);
@SuppressWarnings("resource")
var wrapped = new IsolatedScanner(scanner);
scanner = wrapped;
}
if (usesLocalIterators) {
log.info("Using local iterators");
Expand Down
5 changes: 0 additions & 5 deletions minicluster/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -128,11 +128,6 @@
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<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>
Expand Down
11 changes: 2 additions & 9 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -319,7 +319,7 @@
<dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
<version>1.2</version>
<version>1.3.3</version>
</dependency>
<dependency>
<!-- legacy junit version specified here for dependency convergence -->
Expand Down Expand Up @@ -531,13 +531,6 @@
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
<version>${version.thrift}</version>
<!-- exclude vulnerable lib see https://issues.apache.org/jira/browse/HTTPCLIENT-1803 -->
<exclusions>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.zookeeper</groupId>
Expand Down Expand Up @@ -1039,7 +1032,7 @@
<bannedDependencies>
<excludes>
<!-- we redirect logging to log4j2, so we should have those bridges instead -->
<!-- commons-logging is allowed because log4j-jcl uses it as a dependency -->
<!-- commons-logging is allowed because it natively sends to log4j2 or slf4j -->
<exclude>ch.qos.logback:*</exclude>
<exclude>ch.qos.reload4j:*</exclude>
<exclude>log4j:*</exclude>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@
import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.DeletesSection.SkewedKeyValue;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.ExternalCompactionSection;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.BulkFileColumnFamily;
import org.apache.accumulo.core.security.Authorizations;
Expand Down Expand Up @@ -363,7 +362,8 @@ public Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
BatchScanner scanner =
context.createBatchScanner(DataLevel.USER.metaTable(), Authorizations.EMPTY);
scanner.setRanges(Set.of(ScanServerFileReferenceSection.getRange(),
OldScanServerFileReferenceSection.getRange()));
org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection
.getRange()));
return StreamSupport.stream(scanner.spliterator(), false)
.map(e -> new ScanServerRefTabletFile(e.getKey()));
} catch (TableNotFoundException e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -266,9 +266,9 @@ public DistributionStatisticConfig configure(Meter.Id id,
}
}

// support for org.apache.accumulo.core.metrics.MeterRegistryFactory can be removed in 3.1
@VisibleForTesting
@SuppressWarnings({"deprecation",
"support for org.apache.accumulo.core.metrics.MeterRegistryFactory can be removed in 3.1"})
@SuppressWarnings("deprecation")
static MeterRegistry getRegistryFromFactory(final String factoryName, final ServerContext context)
throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException,
InstantiationException, IllegalAccessException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,7 +242,7 @@ private StatusSummary getCompactorHosts(final ZooReader zooReader, final String
Map<String,Set<String>> hostsByGroups = new TreeMap<>();

// get group names
Result<Integer,Set<String>> queueNodes = readNodeNames(zooReader, zRootPath);
Result<Set<String>> queueNodes = readNodeNames(zooReader, zRootPath);
errors.addAndGet(queueNodes.getErrorCount());
Set<String> queues = new TreeSet<>(queueNodes.getHosts());

Expand All @@ -265,7 +265,7 @@ private StatusSummary getCompactorHosts(final ZooReader zooReader, final String
* @return Result with error count, Set of the node names.
*/
@VisibleForTesting
Result<Integer,Set<String>> readNodeNames(final ZooReader zooReader, final String path) {
Result<Set<String>> readNodeNames(final ZooReader zooReader, final String path) {
Set<String> nodeNames = new TreeSet<>();
final AtomicInteger errorCount = new AtomicInteger(0);
try {
Expand All @@ -290,7 +290,7 @@ Result<Integer,Set<String>> readNodeNames(final ZooReader zooReader, final Strin
* @return Pair with error count, the node data as String.
*/
@VisibleForTesting
Result<Integer,String> readNodeData(final ZooReader zooReader, final String path) {
Result<String> readNodeData(final ZooReader zooReader, final String path) {
try {
byte[] data = zooReader.getData(path);
return new Result<>(0, new String(data, UTF_8));
Expand All @@ -311,7 +311,7 @@ Result<Integer,String> readNodeData(final ZooReader zooReader, final String path
* @return Pair with error count, the data from each node as a String.
*/
@VisibleForTesting
Result<Integer,Set<String>> readAllNodesData(final ZooReader zooReader, final String path) {
Result<Set<String>> readAllNodesData(final ZooReader zooReader, final String path) {
Set<String> hosts = new TreeSet<>();
final AtomicInteger errorCount = new AtomicInteger(0);
try {
Expand Down Expand Up @@ -352,12 +352,12 @@ public static class Opts {
* @param <A> errorCount
* @param <B> hosts
*/
private static class Result<A extends Integer,B> extends Pair<A,B> {
public Result(A errorCount, B hosts) {
private static class Result<B> extends Pair<Integer,B> {
public Result(Integer errorCount, B hosts) {
super(errorCount, hosts);
}

public A getErrorCount() {
public Integer getErrorCount() {
return getFirst();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,8 @@ public void factoryTest() throws Exception {
verify(context, conf);
}

@SuppressWarnings({"deprecation",
"support for org.apache.accumulo.core.metrics.MeterRegistryFactory can be removed in 3.1"})
// 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() {
Expand Down
4 changes: 0 additions & 4 deletions server/compaction-coordinator/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -55,10 +55,6 @@
<groupId>org.apache.accumulo</groupId>
<artifactId>accumulo-start</artifactId>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-1.2-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
Expand Down
4 changes: 0 additions & 4 deletions server/compactor/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -63,10 +63,6 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-1.2-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,9 @@
import org.apache.accumulo.core.metrics.MetricsProducer;
import org.apache.accumulo.manager.Manager;
import org.apache.accumulo.manager.metrics.fate.FateMetrics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class ManagerMetrics {

private final static Logger log = LoggerFactory.getLogger(ManagerMetrics.class);

public static List<MetricsProducer> getProducers(AccumuloConfiguration conf, Manager m) {
ArrayList<MetricsProducer> producers = new ArrayList<>();
@SuppressWarnings("deprecation")
Expand Down
2 changes: 1 addition & 1 deletion server/monitor/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@
<artifactId>slf4j-api</artifactId>
</dependency>
<dependency>
<!-- used by log4j; needed here for compile time linting only; we don't actually use it -->
<!-- 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>
Expand Down
1 change: 1 addition & 0 deletions start/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
<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>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -204,12 +204,13 @@ public boolean accept(Key k, Value v) {
private void runPluginEnvTest(Set<String> expected) throws Exception {
try (var scanner = client.createScanner(tableName)) {
initCalled.set(false);
var csis = new ClientSideIteratorScanner(scanner);
csis.addScanIterator(new IteratorSetting(100, "filter", TestPropFilter.class));
assertEquals(expected,
csis.stream().map(e -> e.getKey().getRowData().toString()).collect(Collectors.toSet()));
// this check is here to ensure the iterator executed client side and not server side
assertTrue(initCalled.get());
try (var csis = new ClientSideIteratorScanner(scanner)) {
csis.addScanIterator(new IteratorSetting(100, "filter", TestPropFilter.class));
assertEquals(expected,
csis.stream().map(e -> e.getKey().getRowData().toString()).collect(Collectors.toSet()));
// this check is here to ensure the iterator executed client side and not server side
assertTrue(initCalled.get());
}
}

// The offline scanner also runs iterators client side, so test its client side access to
Expand Down
4 changes: 0 additions & 4 deletions test/src/main/java/org/apache/accumulo/test/PrintInfoIT.java
Original file line number Diff line number Diff line change
Expand Up @@ -55,13 +55,9 @@
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class PrintInfoIT extends SharedMiniClusterBase {

private static final Logger log = LoggerFactory.getLogger(PrintInfoIT.class);

@BeforeAll
public static void setup() throws Exception {
SharedMiniClusterBase.startMiniCluster();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
import org.apache.accumulo.core.metadata.schema.Ample;
import org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection;
import org.apache.accumulo.core.util.HostAndPort;
import org.apache.accumulo.harness.SharedMiniClusterBase;
import org.apache.accumulo.server.ServerContext;
Expand Down Expand Up @@ -80,7 +79,6 @@ public void testServerContextMethods() {
}

@Test
@SuppressWarnings("deprecation")
public void testOldScanServerRefs() {
HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
UUID serverLockUUID = UUID.randomUUID();
Expand All @@ -97,7 +95,10 @@ public void testOldScanServerRefs() {

// Add old scan server entries
try (BatchWriter writer = ctx.createBatchWriter(Ample.DataLevel.USER.metaTable())) {
String prefix = OldScanServerFileReferenceSection.getRowPrefix();
@SuppressWarnings("deprecation")
String prefix =
org.apache.accumulo.core.metadata.schema.MetadataSchema.OldScanServerFileReferenceSection
.getRowPrefix();
for (String filepath : Stream.of("F0001243.rf", "F0006512.rf")
.map(f -> "hdfs://localhost:8020/accumulo/tables/2a/test_tablet/" + f)
.collect(Collectors.toSet())) {
Expand Down

0 comments on commit a7167da

Please sign in to comment.