Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1546,22 +1546,15 @@ class BeamModulePlugin implements Plugin<Project> {
"PreferJavaTimeOverload",
"NonCanonicalType",
"Slf4jSignOnlyFormat",
"ThreadPriorityCheck",
"UndefinedEquals",
"UnescapedEntity",
"UnrecognisedJavadocTag",
// errorprone 3.2.0+ checks
"DirectInvocationOnMock",
"Finalize",
"JUnitIncompatibleType",
"MockNotUsedInProduction",
"NullableTypeParameter",
"NullableWildcard",
"SuperCallToObjectMethod",
// intended suppressions emerged in newer protobuf versions
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 is only suppression but enabling it will help resolving future issues.

Can we add this comment back

// intended suppressions emerged in newer errorprone versions

or

intended suppressions with justifications

The suppression below this line were preferred for performance/backward compatibility or other reasons that may affect actual functionality and not merely code style issue

In case future "fix" on these suppressions may introduce unwanted change

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

thanks for looking into this , will do.

// For backward compatibility. Public method checked in before this check impl
// Possible use in interface subclasses
"ClassInitializationDeadlock",
// for encoding efficiency and backward compatibility
"EnumOrdinal",
// widely used in non-public methods
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ public void close() {
}

@Override
@SuppressWarnings("Finalize")
protected void finalize() {
destroySynchronized();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public ApplianceShuffleWriter(
}

@Override
@SuppressWarnings("Finalize")
protected void finalize() {
destroy();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,7 @@ private static DebugCapture.Manager initializeAndStartDebugCaptureManager(
return result;
}

@SuppressWarnings("ThreadPriorityCheck")
private static Thread startMemoryMonitorThread(MemoryMonitor memoryMonitor) {
Thread result = new Thread(memoryMonitor);
result.setDaemon(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ protected WindmillServerBase(String host) {
}

@Override
@SuppressWarnings("Finalize")
protected void finalize() {
destroy(nativePointer);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@ public interface CheckpointMark {
*/
void finalizeCheckpoint() throws IOException;

@SuppressWarnings("ClassInitializationDeadlock")
NoopCheckpointMark NOOP_CHECKPOINT_MARK = new NoopCheckpointMark();

/** A checkpoint mark that does nothing when finalized. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -284,6 +284,7 @@ default boolean observesWindow() {
}

/** A descriptor for an optional parameter of the {@link DoFn.ProcessElement} method. */
@SuppressWarnings("ClassInitializationDeadlock")
public abstract static class Parameter {

// Private as no extensions other than those nested here are permitted
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ public final class UnboundedScheduledExecutorService implements ScheduledExecuto
value = "EQ_COMPARETO_USE_OBJECT_EQUALS",
justification =
"Default equals/hashCode is what we want since two scheduled tasks are only equivalent if they point to the same instance.")
@SuppressWarnings("NullableTypeParameter")
final class ScheduledFutureTask<@Nullable @KeyForBottom V> extends FutureTask<V>
implements RunnableScheduledFuture<V> {

Expand Down Expand Up @@ -373,6 +374,7 @@ public void execute(Runnable command) {
}

@Override
@SuppressWarnings("NullableTypeParameter")
public Future<@Nullable ?> submit(Runnable command) {
if (command == null) {
throw new NullPointerException();
Expand All @@ -384,7 +386,7 @@ public void execute(Runnable command) {

@Override
/* Ignore improper flag since FB detects that ScheduledExecutorService can't have nullable V. */
@SuppressWarnings("override.return")
@SuppressWarnings({"override.return", "NullableTypeParameter"})
public <@Nullable @KeyForBottom T> Future<T> submit(Runnable command, T result) {
if (command == null) {
throw new NullPointerException();
Expand All @@ -396,7 +398,7 @@ public void execute(Runnable command) {

@Override
/* Ignore improper flag since FB detects that ScheduledExecutorService can't have nullable V. */
@SuppressWarnings({"override.param", "override.return"})
@SuppressWarnings({"override.param", "override.return", "NullableTypeParameter"})
public <@Nullable @KeyForBottom T> Future<T> submit(Callable<T> command) {
if (command == null) {
throw new NullPointerException();
Expand Down Expand Up @@ -436,6 +438,7 @@ public void execute(Runnable command) {
}

@Override
@SuppressWarnings("NullableTypeParameter")
public ScheduledFuture<@Nullable ?> schedule(Runnable command, long delay, TimeUnit unit) {
if (command == null || unit == null) {
throw new NullPointerException();
Expand All @@ -448,7 +451,7 @@ public void execute(Runnable command) {

@Override
/* Ignore improper flag since FB detects that ScheduledExecutorService can't have nullable V. */
@SuppressWarnings({"override.param", "override.return"})
@SuppressWarnings({"override.param", "override.return", "NullableTypeParameter"})
public <@Nullable @KeyForBottom V> ScheduledFuture<V> schedule(
Callable<V> callable, long delay, TimeUnit unit) {
if (callable == null || unit == null) {
Expand All @@ -460,6 +463,7 @@ public void execute(Runnable command) {
}

@Override
@SuppressWarnings("NullableTypeParameter")
public ScheduledFuture<@Nullable ?> scheduleAtFixedRate(
Runnable command, long initialDelay, long period, TimeUnit unit) {
if (command == null || unit == null) {
Expand All @@ -476,6 +480,7 @@ public void execute(Runnable command) {
}

@Override
@SuppressWarnings("NullableTypeParameter")
public ScheduledFuture<@Nullable ?> scheduleWithFixedDelay(
Runnable command, long initialDelay, long delay, TimeUnit unit) {
if (command == null || unit == null) {
Expand All @@ -491,6 +496,7 @@ public void execute(Runnable command) {
return task;
}

@SuppressWarnings("NullableTypeParameter")
private <@Nullable @KeyForBottom T> void runNowOrScheduleInTheFuture(
ScheduledFutureTask<T> task) {
long nanosToWait = LongMath.saturatedSubtract(task.time, clock.nanoTime());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ public class BeamFnStatusClient implements AutoCloseable {
private final MemoryMonitor memoryMonitor;
private final Cache<?, ?> cache;

@SuppressWarnings("ThreadPriorityCheck")
@SuppressFBWarnings("SC_START_IN_CTOR") // for memory monitor thread
public BeamFnStatusClient(
ApiServiceDescriptor apiServiceDescriptor,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
* href="https://github.com/apache/iceberg/blob/main/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java">Iceberg's
* integration testing</a>
*/
@SuppressWarnings({"OperatorPrecedence", "DefaultCharset"})
@SuppressWarnings({"OperatorPrecedence", "DefaultCharset", "ThreadPriorityCheck"})
Comment thread
kennknowles marked this conversation as resolved.
public class ScriptRunner {

private static final String DEFAULT_DELIMITER = ";";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3848,6 +3848,7 @@ public void populateDisplayData(DisplayData.Builder builder) {
}
}

@SuppressWarnings("NullableTypeParameter")
private static class NullOnlyCoder<@Nullable T> extends AtomicCoder<T> {
@Override
public void encode(T value, OutputStream outStream) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,7 @@ private Pair(
* does not enforce {@link Coder#verifyDeterministic} and defers to the user to determine whether
* to enforce this given the cache implementation.
*/
@SuppressWarnings("NullableTypeParameter")
static <
RequestT,
@Nullable ResponseT,
Expand All @@ -138,6 +139,7 @@ private Pair(
* href="https://redis.io/docs/get-started/faq/">Redis FAQ</a> for more information on important
* considerations when using this method to achieve cache reads.
*/
@SuppressWarnings("NullableTypeParameter")
static <RequestT, @Nullable ResponseT>
PTransform<PCollection<RequestT>, Result<KV<RequestT, @Nullable ResponseT>>> readUsingRedis(
RedisClient client,
Expand Down Expand Up @@ -178,6 +180,7 @@ PTransform<PCollection<KV<RequestT, ResponseT>>, Result<KV<RequestT, ResponseT>>
* href="https://redis.io/docs/get-started/faq/">Redis FAQ</a> for more information on important
* considerations when using this method to achieve cache writes.
*/
@SuppressWarnings("NullableTypeParameter")
static <RequestT, ResponseT>
PTransform<PCollection<KV<RequestT, ResponseT>>, Result<KV<RequestT, ResponseT>>>
writeUsingRedis(
Expand Down Expand Up @@ -217,6 +220,7 @@ Write<RequestT, ResponseT> write(Duration expiry) {
}

/** Reads associated {@link RequestT} {@link ResponseT} using a {@link RedisClient}. */
@SuppressWarnings("NullableTypeParameter")
static class Read<RequestT, @Nullable ResponseT>
implements Caller<RequestT, KV<RequestT, @Nullable ResponseT>>, SetupTeardown {

Expand Down
Loading