Skip to content
Open
4 changes: 4 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,10 @@
<artifactId>auto-service</artifactId>
<optional>true</optional>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.gson</groupId>
<artifactId>gson</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@
import java.util.Set;
import java.util.SortedSet;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.Scanner;
import org.apache.accumulo.core.client.TableNotFoundException;
Expand Down Expand Up @@ -76,6 +78,7 @@

import com.google.common.base.Preconditions;

@NotThreadSafe
class RFileScanner extends ScannerOptions implements Scanner {
Comment on lines +82 to 83
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Scanners are not thread safe objects


private static class RFileScannerEnvironmentImpl extends ClientServiceEnvironmentImpl {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.client.ScannerBase;
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
Expand All @@ -46,6 +48,7 @@
import org.apache.accumulo.core.util.TextUtil;
import org.apache.hadoop.io.Text;

@NotThreadSafe
public class ScannerOptions implements ScannerBase {
Comment on lines +51 to 52
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Scanners are not thread safe


protected List<IterInfo> serverSideIteratorList = Collections.emptyList();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,8 @@
import java.util.function.Supplier;
import java.util.stream.Collectors;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.SampleNotPresentException;
Expand Down Expand Up @@ -355,6 +357,7 @@ private String getTableInfo() {
return context.getPrintableTableInfoFromId(tableId);
}

@NotThreadSafe
private class QueryTask implements Runnable {
Comment on lines +360 to 361
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

QueryTask is not used as/expected to be a thread-safe object


private final String tsLocation;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@
import java.util.function.LongSupplier;
import java.util.function.Supplier;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.rpc.ThriftUtil;
import org.apache.accumulo.core.rpc.clients.ThriftClientTypes;
import org.apache.accumulo.core.util.HostAndPort;
Expand Down Expand Up @@ -577,6 +579,7 @@ public TransportPoolShutdownException(String msg) {
private static final long serialVersionUID = 1L;
}

@NotThreadSafe
private static class CachedTTransport extends TTransport {
Comment on lines +582 to 583
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CachedTTransport is only modified by a single thread at a time (must be reserved()/unreserved())


private final ThriftTransportKey cacheKey;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,13 @@
import java.io.IOException;
import java.io.InputStream;

import javax.annotation.concurrent.NotThreadSafe;

/**
* Reader corresponding to BlockedOutputStream. Expects all data to be in the form of size (int)
* data (size bytes) junk (however many bytes it takes to complete a block)
*/
@NotThreadSafe
public class BlockedInputStream extends InputStream {
Comment on lines +32 to 33
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BlockedInputStream is a wrapper for DataInputStream, which is not thread safe

byte[] array;
// ReadPos is where to start reading
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.util.UtilWaitThread;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -107,6 +109,7 @@ public interface QueueLock {

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

@NotThreadSafe
Comment thread
keith-turner marked this conversation as resolved.
static class ReadLock implements Lock {

QueueLock qlock;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,8 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.bloomfilter.DynamicBloomFilter;
import org.apache.accumulo.core.classloader.ClassLoaderUtil;
import org.apache.accumulo.core.conf.AccumuloConfiguration;
Expand Down Expand Up @@ -346,6 +348,7 @@ public void close() {
}
}

@NotThreadSafe
public static class Reader implements FileSKVIterator {
Comment on lines +351 to 352
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accumulo iterators are not expected to be thread safe


private final BloomFilterLoader bfl;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,15 @@
import java.io.IOException;
import java.io.InputStream;

import javax.annotation.concurrent.NotThreadSafe;

import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;

/**
* This class is like byte array input stream with two differences. It supports seeking and avoids
* synchronization.
*/
@NotThreadSafe
public class SeekableByteArrayInputStream extends InputStream {
Comment on lines 30 to 35
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This class is clearly not meant to be thread safe: "It ... avoids synchronization."


// making this volatile for the following case
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Range;
Expand All @@ -43,6 +45,7 @@
public class MapFileOperations extends FileOperations {
private static final String MSG = "Map files are not supported";

@NotThreadSafe
public static class RangeIterator implements FileSKVIterator {
Comment on lines +48 to 49
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accumulo iterators are not expected to be thread safe


SortedKeyValueIterator<Key,Value> reader;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicBoolean;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.client.SampleNotPresentException;
import org.apache.accumulo.core.client.sample.Sampler;
import org.apache.accumulo.core.client.sample.SamplerConfiguration;
Expand Down Expand Up @@ -557,6 +559,7 @@ public void close() throws IOException {
}
}

@NotThreadSafe
public static class Writer implements FileSKVWriter {
Comment on lines +562 to 563
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

impl clearly expects single threaded access (no volatiles or sync (other than close))


public static final int MAX_CF_IN_DLG = 1000;
Expand Down Expand Up @@ -754,6 +757,7 @@ public long getLength() {
}
}

@NotThreadSafe
private static class LocalityGroupReader extends LocalityGroup implements FileSKVIterator {
Comment on lines +760 to 761
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accumulo iterators are not expected to be thread safe


private final CachableBlockFile.Reader reader;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,13 @@
import java.io.IOException;
import java.io.OutputStream;

import javax.annotation.concurrent.NotThreadSafe;

/**
* A simplified BufferedOutputStream with borrowed buffer, and allow users to see how much data have
* been buffered.
*/
@NotThreadSafe
class SimpleBufferedOutputStream extends FilterOutputStream {
Comment on lines +31 to 32
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The fact that this is a stream and the the impl itself show this is not a thread safe class

protected byte[] buf; // the borrowed buffer
protected int count = 0; // bytes used in buffer.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,16 @@
import java.io.IOException;
import java.io.InputStream;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.hadoop.fs.Seekable;

/**
* BoundedRangeFIleInputStream abstracts a contiguous region of a Hadoop FSDataInputStream as a
* regular input stream. One can create multiple BoundedRangeFileInputStream on top of the same
* FSDataInputStream and they would not interfere with each other.
*/
@NotThreadSafe
public class BoundedRangeFileInputStream extends InputStream {
Comment on lines 28 to 34
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Desc and impl show multiple threads can have their own BoundedRangeFileInputStream for same underlying data, but not that they can share same BoundedRangeFileInputStream.


private volatile boolean closed = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import java.util.TreeSet;
import java.util.concurrent.atomic.AtomicBoolean;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Column;
import org.apache.accumulo.core.data.Key;
Expand All @@ -36,6 +38,7 @@
import org.apache.accumulo.core.iterators.ServerSkippingIterator;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;

@NotThreadSafe
public class ColumnFamilySkippingIterator extends ServerSkippingIterator
implements InterruptibleIterator {
Comment on lines +41 to 43
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accumulo iterators are not expected to be thread safe


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Range;
Expand All @@ -34,6 +36,7 @@
import org.apache.accumulo.core.iterators.ServerWrappingIterator;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;

@NotThreadSafe
public class StatsIterator extends ServerWrappingIterator {
Comment on lines +39 to 40
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accumulo iterators are not expected to be thread safe. The counters here are shared, but those are safely atomic. Internal state "numRead" is not and is not expected to be. deepCopy() shows intended use.


private int numRead = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@
import java.util.stream.Stream;
import java.util.stream.StreamSupport;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.client.AccumuloClient;
import org.apache.accumulo.core.client.BatchScanner;
import org.apache.accumulo.core.client.IsolatedScanner;
Expand Down Expand Up @@ -89,6 +91,7 @@
*/
public class TabletsMetadata implements Iterable<TabletMetadata>, AutoCloseable {

@NotThreadSafe
public static class Builder implements TableRangeOptions, TableOptions, RangeOptions, Options {
Comment on lines +94 to 95
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

impl clearly shows that this is not intended to be thread safe (no volatile, no synchronization)


private final List<Text> families = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;

import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;

/**
* This class automates management of static singletons that maintain state for Accumulo clients.
* Historically, Accumulo client code that used Connector had no control over these singletons. The
Expand Down Expand Up @@ -81,6 +83,8 @@ public enum Mode {
private static List<SingletonService> services;

@VisibleForTesting
@SuppressFBWarnings(value = "AT_NONATOMIC_64BIT_PRIMITIVE",
justification = "only called in static init block and testing, no sync needed")
static void reset() {
reservations = 0;
mode = Mode.CLIENT;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,8 +169,10 @@ public String toString() {
private int maxFilesToCompact;
private double lowestRatio;

@SuppressFBWarnings(value = {"UWF_UNWRITTEN_FIELD", "NP_UNWRITTEN_FIELD"},
justification = "Field is written by Gson")
@SuppressFBWarnings(
value = {"UWF_UNWRITTEN_FIELD", "NP_UNWRITTEN_FIELD", "AT_NONATOMIC_64BIT_PRIMITIVE"},
justification = "UWF_UNWRITTEN_FIELD and NP_UNWRITTEN_FIELD: Field is written by Gson. "
+ "AT_NONATOMIC_64BIT_PRIMITIVE: Fields modified here are initialized once, and read-only after.")
@Override
public void init(InitParameters params) {

Expand Down Expand Up @@ -243,6 +245,8 @@ public void init(InitParameters params) {
}

@SuppressWarnings("removal")
@SuppressFBWarnings(value = "AT_STALE_THREAD_WRITE_OF_PRIMITIVE",
justification = "only called in init")
private void determineMaxFilesToCompact(InitParameters params) {
String fqo = params.getFullyQualifiedOption("maxOpen");
if (!params.getServiceEnvironment().getConfiguration().isSet(fqo)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,8 @@ protected Cipher initialValue() {
};

@Override
@SuppressFBWarnings(value = "AT_STALE_THREAD_WRITE_OF_PRIMITIVE",
justification = "Fields modified here are initialized once, and read-only after.")
public void init(Map<String,String> conf) throws CryptoException {
ensureNotInit();
String keyLocation = Objects.requireNonNull(conf.get(KEY_URI_PROPERTY),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
import java.io.InputStream;
import java.util.Objects;

import javax.annotation.concurrent.NotThreadSafe;

/**
* This class was copied from Guava and modified. If this class was not final in Guava it could have
* been extended. Guava has issue 590 open about this.
Expand All @@ -26,6 +28,7 @@
*
* @author Chris Nokleberg
*/
@NotThreadSafe
public final class CountingInputStream extends FilterInputStream {
Comment on lines +31 to 32
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a stream and the impl shows this is not intended to be thread safe


private long count;
Expand Down
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -781,7 +781,7 @@ under the License.
<includeTests>true</includeTests>
<maxHeap>1024</maxHeap>
<maxRank>16</maxRank>
<omitVisitors>ConstructorThrow,SharedVariableAtomicityDetector</omitVisitors>
<omitVisitors>ConstructorThrow</omitVisitors>
<jvmArgs>-Dcom.overstock.findbugs.ignore=com.google.common.util.concurrent.RateLimiter,com.google.common.hash.Hasher,com.google.common.hash.HashCode,com.google.common.hash.HashFunction,com.google.common.hash.Hashing,com.google.common.cache.Cache,com.google.common.io.CountingOutputStream,com.google.common.io.ByteStreams,com.google.common.cache.LoadingCache,com.google.common.base.Stopwatch,com.google.common.cache.RemovalNotification,com.google.common.util.concurrent.Uninterruptibles,com.google.common.reflect.ClassPath,com.google.common.reflect.ClassPath$ClassInfo,com.google.common.base.Throwables,com.google.common.collect.Iterators</jvmArgs>
<plugins combine.children="append">
<plugin>
Expand Down
4 changes: 4 additions & 0 deletions server/base/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,10 @@
<artifactId>auto-service</artifactId>
<optional>true</optional>
</dependency>
<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
</dependency>
<dependency>
<groupId>com.google.code.gson</groupId>
<artifactId>gson</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,15 @@
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.iterators.IteratorEnvironment;
import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
import org.apache.accumulo.core.iterators.WrappingIterator;

@NotThreadSafe
public class CountingIterator extends WrappingIterator {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

accumulo iterators are not thread safe

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This code is trying to accommodate a multi threaded use case of informing other threads about the number of entries read via a AtomicLong. So not sure if this annotation should be applied here.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe the annotation does make sense from the perspective of its not expected for multiple threads to use an instance of this, even if it happens to export info to other threads.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I didn't see your 2nd comment when I made 0b96b08. Let me know if I should revert the changes to this class, and add back the NotThreadSafe annotation.

Copy link
Copy Markdown
Member Author

@kevinrr888 kevinrr888 Jan 29, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also made an optimization to the next() call in that commit. Scope creep, so maybe can remove that and something like that can be a follow on PR.

n % x === n & (x-1) when x is a power of 2 but bitwise and is much faster

Copy link
Copy Markdown
Contributor

@keith-turner keith-turner Jan 29, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah should probably revert those changes. The code is attempting to avoid accessing main memory by using a non volatile and periodically updating a volatile. Do not want to sync or update main memory for each key/value for the purpose of metrics.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reverted and opened #6095


private long count;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;

import javax.annotation.concurrent.NotThreadSafe;

import org.apache.accumulo.core.client.SampleNotPresentException;
import org.apache.accumulo.core.conf.Property;
import org.apache.accumulo.core.data.Key;
Expand Down Expand Up @@ -386,6 +388,7 @@ private void releaseReaders(KeyExtent tablet, List<FileSKVIterator> readers,

}

@NotThreadSafe
static class FileDataSource implements DataSource {
Comment thread
keith-turner marked this conversation as resolved.

private SortedKeyValueIterator<Key,Value> iter;
Expand Down
Loading