Update to Error Prone 2.4

Most of the changes should be semi-clear why they were made. However, BadImport
may not be as obvious: https://errorprone.info/bugpattern/BadImport . That
impacted classes named Type, Entry, and Factory. Also
PublicContructorForAbstractClass:
https://errorprone.info/bugpattern/PublicConstructorForAbstractClass

The JdkObsolete issue is already resolved but is not yet in a release.
This commit is contained in:
Eric Anderson 2020-08-03 12:51:05 -07:00 committed by Eric Anderson
parent 80d62bfce2
commit e92b2275f9
63 changed files with 162 additions and 148 deletions

View File

@ -30,7 +30,7 @@ import io.grpc.ServerCall.Listener;
import io.grpc.ServerCallHandler;
import io.grpc.ServerInterceptor;
import io.grpc.ServerServiceDefinition;
import io.grpc.ServerStreamTracer.Factory;
import io.grpc.ServerStreamTracer;
import io.grpc.ServerTransportFilter;
import io.grpc.Status;
import io.grpc.alts.internal.AltsProtocolNegotiator;
@ -102,7 +102,7 @@ public final class AltsServerBuilder extends ServerBuilder<AltsServerBuilder> {
/** {@inheritDoc} */
@Override
public AltsServerBuilder addStreamTracerFactory(Factory factory) {
public AltsServerBuilder addStreamTracerFactory(ServerStreamTracer.Factory factory) {
delegate.addStreamTracerFactory(factory);
return this;
}

View File

@ -54,7 +54,7 @@ public final class TsiPeer {
private final String name;
private final T value;
public Property(@Nonnull String name, @Nonnull T value) {
protected Property(@Nonnull String name, @Nonnull T value) {
this.name = name;
this.value = value;
}

View File

@ -22,7 +22,6 @@ import com.google.common.base.Objects;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import javax.annotation.Nullable;
import javax.annotation.concurrent.Immutable;
@ -179,7 +178,7 @@ public final class Attributes {
if (data.size() != that.data.size()) {
return false;
}
for (Entry<Key<?>, Object> e : data.entrySet()) {
for (Map.Entry<Key<?>, Object> e : data.entrySet()) {
if (!that.data.containsKey(e.getKey())) {
return false;
}
@ -202,7 +201,7 @@ public final class Attributes {
@Override
public int hashCode() {
int hashCode = 0;
for (Entry<Key<?>, Object> e : data.entrySet()) {
for (Map.Entry<Key<?>, Object> e : data.entrySet()) {
hashCode += Objects.hashCode(e.getKey(), e.getValue());
}
return hashCode;
@ -262,7 +261,7 @@ public final class Attributes {
*/
public Attributes build() {
if (newdata != null) {
for (Entry<Key<?>, Object> entry : base.data.entrySet()) {
for (Map.Entry<Key<?>, Object> entry : base.data.entrySet()) {
if (!newdata.containsKey(entry.getKey())) {
newdata.put(entry.getKey(), entry.getValue());
}

View File

@ -25,7 +25,6 @@ import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
@ -116,7 +115,7 @@ public final class DecompressorRegistry {
@ExperimentalApi("https://github.com/grpc/grpc-java/issues/1704")
public Set<String> getAdvertisedMessageEncodings() {
Set<String> advertisedDecompressors = new HashSet<>(decompressors.size());
for (Entry<String, DecompressorInfo> entry : decompressors.entrySet()) {
for (Map.Entry<String, DecompressorInfo> entry : decompressors.entrySet()) {
if (entry.getValue().advertised) {
advertisedDecompressors.add(entry.getKey());
}

View File

@ -19,7 +19,6 @@ package io.grpc;
import com.google.common.io.BaseEncoding;
import io.grpc.Metadata.AsciiMarshaller;
import io.grpc.Metadata.BinaryStreamMarshaller;
import io.grpc.Metadata.Key;
import java.nio.charset.Charset;
/**
@ -53,13 +52,13 @@ public final class InternalMetadata {
= Metadata.BASE64_ENCODING_OMIT_PADDING;
@Internal
public static <T> Key<T> keyOf(String name, TrustedAsciiMarshaller<T> marshaller) {
public static <T> Metadata.Key<T> keyOf(String name, TrustedAsciiMarshaller<T> marshaller) {
boolean isPseudo = name != null && !name.isEmpty() && name.charAt(0) == ':';
return Metadata.Key.of(name, isPseudo, marshaller);
}
@Internal
public static <T> Key<T> keyOf(String name, AsciiMarshaller<T> marshaller) {
public static <T> Metadata.Key<T> keyOf(String name, AsciiMarshaller<T> marshaller) {
boolean isPseudo = name != null && !name.isEmpty() && name.charAt(0) == ':';
return Metadata.Key.of(name, isPseudo, marshaller);
}

View File

@ -28,7 +28,7 @@ public interface InternalWithLogId {
* numeric ID that is unique among the instances.
*
* <p>The subclasses of this interface usually want to include the log ID in their {@link
* #toString} results.
* Object#toString} results.
*/
InternalLogId getLogId();
}

View File

@ -753,14 +753,14 @@ public final class Metadata {
}
/**
* @return The original name used to create this key.
* Returns the original name used to create this key.
*/
public final String originalName() {
return originalName;
}
/**
* @return The normalized name for this key.
* Returns the normalized name for this key.
*/
public final String name() {
return name;
@ -870,7 +870,7 @@ public final class Metadata {
}
}
/** A binary key for values which should be serialized lazily to {@Link InputStream}s. */
/** A binary key for values which should be serialized lazily to {@link InputStream}s. */
private static class LazyStreamBinaryKey<T> extends Key<T> {
private final BinaryStreamMarshaller<T> marshaller;

View File

@ -30,7 +30,6 @@ import static org.junit.Assert.fail;
import com.google.common.collect.Lists;
import com.google.common.io.ByteStreams;
import io.grpc.Metadata.Key;
import io.grpc.internal.GrpcUtil;
import java.io.ByteArrayInputStream;
import java.io.IOException;
@ -115,9 +114,9 @@ public class MetadataTest {
private static final byte[] LANCE_BYTES = LANCE.getBytes(US_ASCII);
private static final Metadata.Key<Fish> KEY = Metadata.Key.of("test-bin", FISH_MARSHALLER);
private static final Metadata.Key<Fish> KEY_STREAMED =
Key.of("streamed-bin", FISH_STREAM_MARSHALLER);
Metadata.Key.of("streamed-bin", FISH_STREAM_MARSHALLER);
private static final Metadata.Key<Fish> KEY_IMMUTABLE =
Key.of("immutable-bin", IMMUTABLE_FISH_MARSHALLER);
Metadata.Key.of("immutable-bin", IMMUTABLE_FISH_MARSHALLER);
@Test
public void noPseudoHeaders() {
@ -348,7 +347,7 @@ public class MetadataTest {
public void removeIgnoresMissingValue() {
Metadata m = new Metadata();
// Any key will work.
Key<String> key = GrpcUtil.USER_AGENT_KEY;
Metadata.Key<String> key = GrpcUtil.USER_AGENT_KEY;
boolean success = m.remove(key, "agent");
assertFalse(success);
@ -358,7 +357,7 @@ public class MetadataTest {
public void removeAllIgnoresMissingValue() {
Metadata m = new Metadata();
// Any key will work.
Key<String> key = GrpcUtil.USER_AGENT_KEY;
Metadata.Key<String> key = GrpcUtil.USER_AGENT_KEY;
Iterable<String> removed = m.removeAll(key);
assertNull(removed);
@ -366,9 +365,9 @@ public class MetadataTest {
@Test
public void keyEqualsHashNameWorks() {
Key<?> k1 = Key.of("case", Metadata.ASCII_STRING_MARSHALLER);
Metadata.Key<?> k1 = Metadata.Key.of("case", Metadata.ASCII_STRING_MARSHALLER);
Key<?> k2 = Key.of("CASE", Metadata.ASCII_STRING_MARSHALLER);
Metadata.Key<?> k2 = Metadata.Key.of("CASE", Metadata.ASCII_STRING_MARSHALLER);
assertEquals(k1, k1);
assertNotEquals(k1, null);
assertNotEquals(k1, new Object(){});
@ -383,7 +382,7 @@ public class MetadataTest {
@Test
public void invalidKeyName() {
try {
Key.of("io.grpc/key1", Metadata.ASCII_STRING_MARSHALLER);
Metadata.Key.of("io.grpc/key1", Metadata.ASCII_STRING_MARSHALLER);
fail("Should have thrown");
} catch (IllegalArgumentException e) {
assertEquals("Invalid character '/' in key name 'io.grpc/key1'", e.getMessage());
@ -512,7 +511,8 @@ public class MetadataTest {
}
}
private static <T> Key<T> copyKey(Key<T> key, Metadata.BinaryStreamMarshaller<T> marshaller) {
return Key.of(key.originalName(), marshaller);
private static <T> Metadata.Key<T> copyKey(
Metadata.Key<T> key, Metadata.BinaryStreamMarshaller<T> marshaller) {
return Metadata.Key.of(key.originalName(), marshaller);
}
}

View File

@ -137,6 +137,7 @@ public abstract class AbstractBenchmark {
* sudo ip addr add dev lo 127.127.127.127/32 label lo:benchmark
* </pre>
*/
@SuppressWarnings("JdkObsolete") // No choice but to use Enumeration
private static InetAddress buildBenchmarkAddr() {
InetAddress tmp = null;
try {
@ -180,7 +181,7 @@ public abstract class AbstractBenchmark {
private MethodDescriptor<ByteBuf, ByteBuf> flowControlledStreaming;
protected ManagedChannel[] channels;
public AbstractBenchmark() {
protected AbstractBenchmark() {
}
/**

View File

@ -295,7 +295,7 @@ subprojects {
if (rootProject.properties.get('errorProne', true)) {
dependencies {
errorprone 'com.google.errorprone:error_prone_core:2.3.4'
errorprone 'com.google.errorprone:error_prone_core:2.4.0'
errorproneJavac 'com.google.errorprone:javac:9+181-r4173-1'
}
} else {
@ -373,6 +373,8 @@ subprojects {
options.errorprone.check("UnnecessaryAnonymousClass", CheckSeverity.OFF)
// This project targets Java 7 (no time.Duration class)
options.errorprone.check("PreferJavaTimeOverload", CheckSeverity.OFF)
// The warning fails to provide a source location
options.errorprone.check("MissingSummary", CheckSeverity.OFF)
}
compileTestJava {
// LinkedList doesn't hurt much in tests and has lots of usages

View File

@ -34,7 +34,6 @@ import io.grpc.StreamTracer;
import io.opencensus.trace.BlankSpan;
import io.opencensus.trace.EndSpanOptions;
import io.opencensus.trace.MessageEvent;
import io.opencensus.trace.MessageEvent.Type;
import io.opencensus.trace.Span;
import io.opencensus.trace.SpanContext;
import io.opencensus.trace.Status;
@ -283,14 +282,14 @@ final class CensusTracingModule {
public void outboundMessageSent(
int seqNo, long optionalWireSize, long optionalUncompressedSize) {
recordMessageEvent(
span, Type.SENT, seqNo, optionalWireSize, optionalUncompressedSize);
span, MessageEvent.Type.SENT, seqNo, optionalWireSize, optionalUncompressedSize);
}
@Override
public void inboundMessageRead(
int seqNo, long optionalWireSize, long optionalUncompressedSize) {
recordMessageEvent(
span, Type.RECEIVED, seqNo, optionalWireSize, optionalUncompressedSize);
span, MessageEvent.Type.RECEIVED, seqNo, optionalWireSize, optionalUncompressedSize);
}
}
@ -349,14 +348,14 @@ final class CensusTracingModule {
public void outboundMessageSent(
int seqNo, long optionalWireSize, long optionalUncompressedSize) {
recordMessageEvent(
span, Type.SENT, seqNo, optionalWireSize, optionalUncompressedSize);
span, MessageEvent.Type.SENT, seqNo, optionalWireSize, optionalUncompressedSize);
}
@Override
public void inboundMessageRead(
int seqNo, long optionalWireSize, long optionalUncompressedSize) {
recordMessageEvent(
span, Type.RECEIVED, seqNo, optionalWireSize, optionalUncompressedSize);
span, MessageEvent.Type.RECEIVED, seqNo, optionalWireSize, optionalUncompressedSize);
}
}

View File

@ -84,7 +84,6 @@ import io.opencensus.tags.TagValue;
import io.opencensus.trace.BlankSpan;
import io.opencensus.trace.EndSpanOptions;
import io.opencensus.trace.MessageEvent;
import io.opencensus.trace.MessageEvent.Type;
import io.opencensus.trace.Span;
import io.opencensus.trace.SpanBuilder;
import io.opencensus.trace.SpanContext;
@ -542,11 +541,13 @@ public class CensusModulesTest {
inOrder.verify(spyClientSpan, times(3)).addMessageEvent(messageEventCaptor.capture());
List<MessageEvent> events = messageEventCaptor.getAllValues();
assertEquals(
MessageEvent.builder(Type.SENT, 0).setCompressedMessageSize(882).build(), events.get(0));
MessageEvent.builder(MessageEvent.Type.SENT, 0).setCompressedMessageSize(882).build(),
events.get(0));
assertEquals(
MessageEvent.builder(Type.SENT, 1).setUncompressedMessageSize(27).build(), events.get(1));
MessageEvent.builder(MessageEvent.Type.SENT, 1).setUncompressedMessageSize(27).build(),
events.get(1));
assertEquals(
MessageEvent.builder(Type.RECEIVED, 0)
MessageEvent.builder(MessageEvent.Type.RECEIVED, 0)
.setCompressedMessageSize(255)
.setUncompressedMessageSize(90)
.build(),
@ -1065,11 +1066,13 @@ public class CensusModulesTest {
inOrder.verify(spyServerSpan, times(3)).addMessageEvent(messageEventCaptor.capture());
List<MessageEvent> events = messageEventCaptor.getAllValues();
assertEquals(
MessageEvent.builder(Type.SENT, 0).setCompressedMessageSize(882).build(), events.get(0));
MessageEvent.builder(MessageEvent.Type.SENT, 0).setCompressedMessageSize(882).build(),
events.get(0));
assertEquals(
MessageEvent.builder(Type.SENT, 1).setUncompressedMessageSize(27).build(), events.get(1));
MessageEvent.builder(MessageEvent.Type.SENT, 1).setUncompressedMessageSize(27).build(),
events.get(1));
assertEquals(
MessageEvent.builder(Type.RECEIVED, 0)
MessageEvent.builder(MessageEvent.Type.RECEIVED, 0)
.setCompressedMessageSize(255)
.setUncompressedMessageSize(90)
.build(),

View File

@ -16,7 +16,6 @@
package io.grpc;
import io.grpc.Context.Key;
import java.util.concurrent.TimeUnit;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@ -30,7 +29,7 @@ import org.openjdk.jmh.annotations.State;
@State(Scope.Benchmark)
public class AttachDetachBenchmark {
private final Key<Integer> key = Context.keyWithDefault("key", 9999);
private final Context.Key<Integer> key = Context.keyWithDefault("key", 9999);
private final Context cu = Context.current().withValue(key, 8888);
/**

View File

@ -943,6 +943,8 @@ public class Context {
*/
public interface CancellationListener {
/**
* Notifies that a context was cancelled.
*
* @param context the newly cancelled context.
*/
void cancelled(Context context);

View File

@ -75,7 +75,7 @@ public abstract class AbstractClientStream extends AbstractStream
* @param endOfStream {@code true} if this is the last frame; {@code flush} is guaranteed to be
* {@code true} if this is {@code true}
* @param flush {@code true} if more data may not be arriving soon
* @Param numMessages the number of messages this series of frames represents, must be >= 0.
* @param numMessages the number of messages this series of frames represents, must be >= 0.
*/
void writeFrame(
@Nullable WritableBuffer frame, boolean endOfStream, boolean flush, int numMessages);

View File

@ -19,7 +19,7 @@ package io.grpc.internal;
import java.util.concurrent.atomic.AtomicLong;
/**
* An implementation of {@link LongCounter} that is just an {@Link AtomicLong}.
* An implementation of {@link LongCounter} that is just an {@link AtomicLong}.
*/
final class AtomicLongCounter implements LongCounter {
private final AtomicLong counter = new AtomicLong();

View File

@ -25,7 +25,7 @@ public interface BackoffPolicy {
}
/**
* @return The number of nanoseconds to wait.
* Returns the number of nanoseconds to wait.
*/
long nextBackoffNanos();
}

View File

@ -63,10 +63,10 @@ public interface ClientTransportFactory extends Closeable {
void close();
/**
* Options passed to {@link #newClientTransport(SocketAddress, ClientTransportOptions)}. Although
* it is safe to save this object if received, it is generally expected that the useful fields are
* copied and then the options object is discarded. This allows using {@code final} for those
* fields as well as avoids retaining unused objects contained in the options.
* Options passed to {@link #newClientTransport}. Although it is safe to save this object if
* received, it is generally expected that the useful fields are copied and then the options
* object is discarded. This allows using {@code final} for those fields as well as avoids
* retaining unused objects contained in the options.
*/
final class ClientTransportOptions {
private ChannelLogger channelLogger;

View File

@ -26,7 +26,7 @@ abstract class ContextRunnable implements Runnable {
private final Context context;
public ContextRunnable(Context context) {
protected ContextRunnable(Context context) {
this.context = context;
}

View File

@ -46,7 +46,6 @@ import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.Executor;
@ -488,7 +487,7 @@ public class DnsNameResolver extends NameResolver {
@VisibleForTesting
static Map<String, ?> maybeChooseServiceConfig(
Map<String, ?> choice, Random random, String hostname) {
for (Entry<String, ?> entry : choice.entrySet()) {
for (Map.Entry<String, ?> entry : choice.entrySet()) {
Verify.verify(SERVICE_CONFIG_CHOICE_KEYS.contains(entry.getKey()), "Bad key: %s", entry);
}

View File

@ -37,7 +37,7 @@ public abstract class ForwardingReadableBuffer implements ReadableBuffer {
*
* @param buf the underlying buffer
*/
public ForwardingReadableBuffer(ReadableBuffer buf) {
protected ForwardingReadableBuffer(ReadableBuffer buf) {
this.buf = Preconditions.checkNotNull(buf, "buf");
}

View File

@ -194,6 +194,8 @@ final class JndiResourceResolverFactory implements DnsNameResolver.ResourceResol
@VisibleForTesting
@IgnoreJRERequirement
// Hashtable is required. https://github.com/google/error-prone/issues/1766
@SuppressWarnings("JdkObsolete")
// javax.naming.* is only loaded reflectively and is never loaded for Android
// The lint issue id is supposed to be "InvalidPackage" but it doesn't work, don't know why.
// Use "all" as the lint issue id to suppress all types of lint error.
@ -205,7 +207,6 @@ final class JndiResourceResolverFactory implements DnsNameResolver.ResourceResol
String[] rrType = new String[]{recordType};
List<String> records = new ArrayList<>();
@SuppressWarnings("JdkObsolete")
Hashtable<String, String> env = new Hashtable<>();
env.put("com.sun.jndi.ldap.connect.timeout", "5000");
env.put("com.sun.jndi.ldap.read.timeout", "5000");

View File

@ -469,8 +469,8 @@ abstract class RetriableStream<ReqT> implements ClientStream {
}
/**
* Do not use it directly. Use {@link #sendMessage(ReqT)} instead because we don't use InputStream
* for buffering.
* Do not use it directly. Use {@link #sendMessage(Object)} instead because we don't use
* InputStream for buffering.
*/
@Override
public final void writeMessage(InputStream message) {

View File

@ -37,7 +37,6 @@ final class ServiceConfigState {
/**
* @param defaultServiceConfig The initial service config, or {@code null} if absent.
* @param lookUpServiceConfig {@code true} if service config updates might occur.
* @param syncCtx The synchronization context that this is accessed from.
*/
ServiceConfigState(
@Nullable ManagedChannelServiceConfig defaultServiceConfig,

View File

@ -23,7 +23,7 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertSame;
import com.google.common.collect.Iterables;
import io.grpc.ServerStreamTracer.Factory;
import io.grpc.ServerStreamTracer;
import io.grpc.internal.FakeClock;
import io.grpc.internal.ObjectPool;
import io.grpc.internal.SharedResourcePool;
@ -55,8 +55,8 @@ public class InProcessServerBuilderTest {
@Test
public void scheduledExecutorService_default() {
InProcessServerBuilder builder = InProcessServerBuilder.forName("foo");
InProcessServer server =
Iterables.getOnlyElement(builder.buildTransportServers(new ArrayList<Factory>()));
InProcessServer server = Iterables.getOnlyElement(
builder.buildTransportServers(new ArrayList<ServerStreamTracer.Factory>()));
ObjectPool<ScheduledExecutorService> scheduledExecutorServicePool =
server.getScheduledExecutorServicePool();
@ -80,8 +80,8 @@ public class InProcessServerBuilderTest {
InProcessServerBuilder builder1 = builder.scheduledExecutorService(scheduledExecutorService);
assertSame(builder, builder1);
InProcessServer server =
Iterables.getOnlyElement(builder1.buildTransportServers(new ArrayList<Factory>()));
InProcessServer server = Iterables.getOnlyElement(
builder1.buildTransportServers(new ArrayList<ServerStreamTracer.Factory>()));
ObjectPool<ScheduledExecutorService> scheduledExecutorServicePool =
server.getScheduledExecutorServicePool();

View File

@ -42,7 +42,6 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture;
import io.grpc.Attributes;
import io.grpc.Attributes.Key;
import io.grpc.CallOptions;
import io.grpc.ClientCall;
import io.grpc.ClientStreamTracer;
@ -1007,8 +1006,8 @@ public class ClientCallImplTest {
ClientCallImpl<Void, Void> call = new ClientCallImpl<>(
method, MoreExecutors.directExecutor(), baseCallOptions, clientStreamProvider,
deadlineCancellationExecutor, channelCallTracer);
Attributes attrs =
Attributes.newBuilder().set(Key.<String>create("fake key"), "fake value").build();
Attributes attrs = Attributes.newBuilder().set(
Attributes.Key.<String>create("fake key"), "fake value").build();
when(stream.getAttributes()).thenReturn(attrs);
assertNotEquals(attrs, call.getAttributes());

View File

@ -33,7 +33,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import io.grpc.Attributes;
import io.grpc.Attributes.Key;
import io.grpc.Codec;
import io.grpc.DecompressorRegistry;
import io.grpc.Metadata;
@ -184,7 +183,7 @@ public class DelayedStreamTest {
@Test
public void setStream_getAttributes() {
Attributes attributes =
Attributes.newBuilder().set(Key.<String>create("fakeKey"), "fakeValue").build();
Attributes.newBuilder().set(Attributes.Key.<String>create("fakeKey"), "fakeValue").build();
when(realStream.getAttributes()).thenReturn(attributes);
stream.start(listener);

View File

@ -23,7 +23,6 @@ import static org.mockito.Mockito.mock;
import io.grpc.ChannelLogger;
import io.grpc.NameResolver;
import io.grpc.NameResolver.Factory;
import io.grpc.NameResolver.ServiceConfigParser;
import io.grpc.ProxyDetector;
import io.grpc.SynchronizationContext;
@ -104,7 +103,7 @@ public class ManagedChannelImplGetNameResolverTest {
@Test
public void validTargetNoResovler() {
Factory nameResolverFactory = new NameResolver.Factory() {
NameResolver.Factory nameResolverFactory = new NameResolver.Factory() {
@Override
public NameResolver newNameResolver(URI targetUri, NameResolver.Args args) {
return null;
@ -125,7 +124,7 @@ public class ManagedChannelImplGetNameResolverTest {
}
private void testValidTarget(String target, String expectedUriString, URI expectedUri) {
Factory nameResolverFactory = new FakeNameResolverFactory(expectedUri.getScheme());
NameResolver.Factory nameResolverFactory = new FakeNameResolverFactory(expectedUri.getScheme());
FakeNameResolver nameResolver = (FakeNameResolver) ManagedChannelImpl.getNameResolver(
target, nameResolverFactory, NAMERESOLVER_ARGS);
assertNotNull(nameResolver);
@ -134,7 +133,7 @@ public class ManagedChannelImplGetNameResolverTest {
}
private void testInvalidTarget(String target) {
Factory nameResolverFactory = new FakeNameResolverFactory("dns");
NameResolver.Factory nameResolverFactory = new FakeNameResolverFactory("dns");
try {
FakeNameResolver nameResolver = (FakeNameResolver) ManagedChannelImpl.getNameResolver(

View File

@ -47,7 +47,7 @@ public final class TestUtils {
public abstract static class StandardLoadBalancerProvider extends LoadBalancerProvider {
private final String policyName;
public StandardLoadBalancerProvider(String policyName) {
protected StandardLoadBalancerProvider(String policyName) {
this.policyName = policyName;
}

View File

@ -31,7 +31,6 @@ import com.google.common.io.BaseEncoding;
import io.grpc.InternalMetadata;
import io.grpc.Metadata;
import io.grpc.Metadata.BinaryMarshaller;
import io.grpc.Metadata.Key;
import java.util.Arrays;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -58,11 +57,14 @@ public class TransportFrameUtilTest {
}
};
private static final Key<String> PLAIN_STRING = Key.of("plainstring", ASCII_STRING_MARSHALLER);
private static final Key<String> BINARY_STRING = Key.of("string-bin", UTF8_STRING_MARSHALLER);
private static final Key<String> BINARY_STRING_WITHOUT_SUFFIX =
Key.of("string", ASCII_STRING_MARSHALLER);
private static final Key<byte[]> BINARY_BYTES = Key.of("bytes-bin", BINARY_BYTE_MARSHALLER);
private static final Metadata.Key<String> PLAIN_STRING =
Metadata.Key.of("plainstring", ASCII_STRING_MARSHALLER);
private static final Metadata.Key<String> BINARY_STRING =
Metadata.Key.of("string-bin", UTF8_STRING_MARSHALLER);
private static final Metadata.Key<String> BINARY_STRING_WITHOUT_SUFFIX =
Metadata.Key.of("string", ASCII_STRING_MARSHALLER);
private static final Metadata.Key<byte[]> BINARY_BYTES =
Metadata.Key.of("bytes-bin", BINARY_BYTE_MARSHALLER);
@Test
public void testToHttp2Headers() {
@ -86,7 +88,7 @@ public class TransportFrameUtilTest {
@Test(expected = IllegalArgumentException.class)
public void binaryHeaderWithoutSuffix() {
Key.of("plainstring", UTF8_STRING_MARSHALLER);
Metadata.Key.of("plainstring", UTF8_STRING_MARSHALLER);
}
@Test

View File

@ -28,7 +28,6 @@ import io.grpc.lb.v1.ClientStatsPerToken;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
@ -115,7 +114,7 @@ final class GrpclbClientLoadRecorder extends ClientStreamTracer.Factory {
callsDroppedPerToken = new HashMap<>(localCallsDroppedPerToken.size());
}
}
for (Entry<String, LongHolder> entry : localCallsDroppedPerToken.entrySet()) {
for (Map.Entry<String, LongHolder> entry : localCallsDroppedPerToken.entrySet()) {
statsBuilder.addCallsFinishedWithDrop(
ClientStatsPerToken.newBuilder()
.setLoadBalanceToken(entry.getKey())

View File

@ -71,7 +71,6 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@ -424,7 +423,7 @@ final class GrpclbState {
newBackendList.add(entry);
}
// Close Subchannels whose addresses have been delisted
for (Entry<List<EquivalentAddressGroup>, Subchannel> entry : subchannels.entrySet()) {
for (Map.Entry<List<EquivalentAddressGroup>, Subchannel> entry : subchannels.entrySet()) {
List<EquivalentAddressGroup> eagList = entry.getKey();
if (!newSubchannelMap.containsKey(eagList)) {
returnSubchannelToPool(entry.getValue());

View File

@ -193,7 +193,7 @@ public abstract class AbstractInteropTest {
/**
* Constructor for tests.
*/
public AbstractInteropTest() {
protected AbstractInteropTest() {
TestRule timeout = Timeout.seconds(60);
try {
timeout = new DisableOnDebug(timeout);

View File

@ -72,8 +72,8 @@ public final class GrpclbLongLivedAffinityTestClient {
private long rpcIntermissionSeconds = 1;
private long totalTestSeconds = 60;
protected ManagedChannel channel;
protected TestServiceGrpc.TestServiceBlockingStub blockingStub;
ManagedChannel channel;
TestServiceGrpc.TestServiceBlockingStub blockingStub;
private void parseArgs(String[] args) {
boolean usage = false;

View File

@ -85,9 +85,9 @@ public final class Http2Client {
private Tester tester = new Tester();
private ListeningExecutorService threadpool;
protected ManagedChannel channel;
protected TestServiceGrpc.TestServiceBlockingStub blockingStub;
protected TestServiceGrpc.TestServiceStub asyncStub;
ManagedChannel channel;
TestServiceGrpc.TestServiceBlockingStub blockingStub;
TestServiceGrpc.TestServiceStub asyncStub;
private void parseArgs(String[] args) {
boolean usage = false;

View File

@ -20,7 +20,7 @@ import io.netty.handler.codec.Headers;
import io.netty.handler.codec.http2.Http2Headers;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.Map;
import java.util.Set;
abstract class AbstractHttp2Headers implements Http2Headers {
@ -496,7 +496,7 @@ abstract class AbstractHttp2Headers implements Http2Headers {
}
@Override
public Iterator<Entry<CharSequence, CharSequence>> iterator() {
public Iterator<Map.Entry<CharSequence, CharSequence>> iterator() {
throw new UnsupportedOperationException();
}

View File

@ -35,7 +35,7 @@ public abstract class GrpcHttp2ConnectionHandler extends Http2ConnectionHandler
@Nullable
protected final ChannelPromise channelUnused;
public GrpcHttp2ConnectionHandler(
protected GrpcHttp2ConnectionHandler(
ChannelPromise channelUnused,
Http2ConnectionDecoder decoder,
Http2ConnectionEncoder encoder,

View File

@ -19,7 +19,7 @@ package io.grpc.netty;
import io.netty.handler.codec.http2.Http2Headers;
import io.netty.util.AsciiString;
import java.util.Iterator;
import java.util.Map.Entry;
import java.util.Map;
import java.util.NoSuchElementException;
/**
@ -77,7 +77,7 @@ final class GrpcHttp2OutboundHeaders extends AbstractHttp2Headers {
}
@Override
public Iterator<Entry<CharSequence, CharSequence>> iterator() {
public Iterator<Map.Entry<CharSequence, CharSequence>> iterator() {
return new Itr();
}
@ -86,8 +86,8 @@ final class GrpcHttp2OutboundHeaders extends AbstractHttp2Headers {
return (normalHeaders.length + preHeaders.length) / 2;
}
private class Itr implements Entry<CharSequence, CharSequence>,
Iterator<Entry<CharSequence, CharSequence>> {
private class Itr implements Map.Entry<CharSequence, CharSequence>,
Iterator<Map.Entry<CharSequence, CharSequence>> {
private int idx;
private AsciiString[] current = preHeaders.length != 0 ? preHeaders : normalHeaders;
private AsciiString key;
@ -104,7 +104,7 @@ final class GrpcHttp2OutboundHeaders extends AbstractHttp2Headers {
* speeds before and after.
*/
@Override
public Entry<CharSequence, CharSequence> next() {
public Map.Entry<CharSequence, CharSequence> next() {
if (hasNext()) {
key = current[idx];
value = current[idx + 1];
@ -144,7 +144,7 @@ final class GrpcHttp2OutboundHeaders extends AbstractHttp2Headers {
public String toString() {
StringBuilder builder = new StringBuilder(getClass().getSimpleName()).append('[');
String separator = "";
for (Entry<CharSequence, CharSequence> e : this) {
for (Map.Entry<CharSequence, CharSequence> e : this) {
CharSequence name = e.getKey();
CharSequence value = e.getValue();
builder.append(separator);

View File

@ -231,7 +231,7 @@ class NettyClientStream extends AbstractClientStream {
private Http2Stream http2Stream;
private Tag tag;
public TransportState(
protected TransportState(
NettyClientHandler handler,
EventLoop eventLoop,
int maxMessageSize,

View File

@ -40,7 +40,7 @@ interface ProtocolNegotiator {
/**
* Releases resources held by this negotiator. Called when the Channel transitions to terminated
* or when InternalServer is shutdown (depending on client or server). That means handlers
* returned by {@link #newHandler()} can outlive their parent negotiator on server-side, but not
* returned by {@link #newHandler} can outlive their parent negotiator on server-side, but not
* on client-side.
*/
void close();

View File

@ -56,7 +56,6 @@ import java.lang.reflect.Constructor;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.UnresolvedAddressException;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.logging.Level;
@ -455,7 +454,7 @@ class Utils {
b.setSocketOptionTimeoutMillis(timeoutMillis);
}
for (Entry<ChannelOption<?>, Object> opt : config.getOptions().entrySet()) {
for (Map.Entry<ChannelOption<?>, Object> opt : config.getOptions().entrySet()) {
ChannelOption<?> key = opt.getKey();
// Constants are pooled, so there should only be one instance of each constant
if (key.equals(SO_LINGER) || key.equals(SO_TIMEOUT)) {
@ -470,7 +469,7 @@ class Utils {
= NettySocketSupport.getNativeSocketOptions(channel);
if (nativeOptions != null) {
b.setTcpInfo(nativeOptions.tcpInfo); // may be null
for (Entry<String, String> entry : nativeOptions.otherInfo.entrySet()) {
for (Map.Entry<String, String> entry : nativeOptions.otherInfo.entrySet()) {
b.addOption(entry.getKey(), entry.getValue());
}
}

View File

@ -62,6 +62,7 @@ import io.grpc.internal.ClientTransport;
import io.grpc.internal.ClientTransport.PingCallback;
import io.grpc.internal.GrpcUtil;
import io.grpc.internal.KeepAliveManager;
import io.grpc.internal.ManagedClientTransport;
import io.grpc.internal.StatsTraceContext;
import io.grpc.internal.StreamListener;
import io.grpc.internal.TransportTracer;
@ -129,7 +130,7 @@ public class NettyClientHandlerTest extends NettyHandlerTestBase<NettyClientHand
@Rule
public TestName testNameRule = new TestName();
@Mock
private NettyClientTransport.Listener listener;
private ManagedClientTransport.Listener listener;
@Mock
private ClientStreamListener streamListener;

View File

@ -21,7 +21,7 @@ import static org.mockito.Mockito.when;
import com.google.common.collect.ImmutableList;
import com.google.common.truth.Truth;
import io.grpc.ServerStreamTracer.Factory;
import io.grpc.ServerStreamTracer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.local.LocalServerChannel;
import io.netty.handler.ssl.SslContext;
@ -47,7 +47,8 @@ public class NettyServerBuilderTest {
@Test
public void createMultipleServers() {
builder.addListenAddress(new InetSocketAddress(8081));
List<NettyServer> servers = builder.buildTransportServers(ImmutableList.<Factory>of());
List<NettyServer> servers =
builder.buildTransportServers(ImmutableList.<ServerStreamTracer.Factory>of());
Truth.assertThat(servers).hasSize(2);
}

View File

@ -75,6 +75,7 @@ import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URI;
import java.util.Collections;
import java.util.Deque;
import java.util.EnumMap;
import java.util.HashMap;
import java.util.Iterator;
@ -188,7 +189,7 @@ class OkHttpClientTransport implements ConnectionClientTransport, TransportExcep
private int maxConcurrentStreams = 0;
@SuppressWarnings("JdkObsolete") // Usage is bursty; want low memory usage when empty
@GuardedBy("lock")
private final LinkedList<OkHttpClientStream> pendingStreams = new LinkedList<>();
private final Deque<OkHttpClientStream> pendingStreams = new LinkedList<>();
private final ConnectionSpec connectionSpec;
private FrameWriter testFrameWriter;
private ScheduledExecutorService scheduler;

View File

@ -85,7 +85,7 @@ final class OkHttpTlsUpgrader {
* @see <a href="https://www.ietf.org/rfc/rfc2732.txt">RFC2732</a>
* @see <a href="https://tools.ietf.org/html/rfc5280#section-4.2.1.6">RFC5280</a>
*
* @return {@param host} in a form consistent with X509 certificates
* @return {@code host} in a form consistent with X509 certificates
*/
@VisibleForTesting
static String canonicalizeHost(String host) {

View File

@ -177,7 +177,9 @@ public class Platform {
trafficStatsTagSocket = trafficStats.getMethod("tagSocket", Socket.class);
trafficStatsUntagSocket = trafficStats.getMethod("untagSocket", Socket.class);
} catch (ClassNotFoundException ignored) {
// On older Android
} catch (NoSuchMethodException ignored) {
// On older Android
}
TlsExtensionType tlsExtensionType;
@ -244,10 +246,15 @@ public class Platform {
});
return new JdkAlpnPlatform(sslProvider, setApplicationProtocols, getApplicationProtocol);
} catch (NoSuchAlgorithmException ignored) {
// On older Java
} catch (KeyManagementException ignored) {
// On older Java
} catch (PrivilegedActionException ignored) {
// On older Java
} catch (IllegalAccessException ignored) {
// On older Java
} catch (InvocationTargetException ignored) {
// On older Java
}
// Find Jetty's ALPN extension for OpenJDK.
@ -264,7 +271,9 @@ public class Platform {
putMethod, getMethod, removeMethod, clientProviderClass, serverProviderClass,
sslProvider);
} catch (ClassNotFoundException ignored) {
// No Jetty ALPN
} catch (NoSuchMethodException ignored) {
// Weird Jetty ALPN
}
// TODO(ericgribkoff) Return null here
@ -512,7 +521,9 @@ public class Platform {
removeMethod.invoke(null, sslSocket);
} catch (IllegalAccessException ignored) {
throw new AssertionError();
} catch (InvocationTargetException ignored) {
} catch (InvocationTargetException ex) {
// This would be very surprising and there's not much to do about it
logger.log(Level.FINE, "Failed to remove SSLSocket from Jetty ALPN", ex);
}
}

View File

@ -74,6 +74,7 @@ public final class Util {
} catch (RuntimeException rethrown) {
throw rethrown;
} catch (Exception ignored) {
// The method is defined to ignore checked exceptions
}
}
}
@ -91,6 +92,7 @@ public final class Util {
} catch (RuntimeException rethrown) {
throw rethrown;
} catch (Exception ignored) {
// The method is defined to ignore checked exceptions
}
}
}
@ -106,6 +108,7 @@ public final class Util {
} catch (RuntimeException rethrown) {
throw rethrown;
} catch (Exception ignored) {
// The method is defined to ignore checked exceptions
}
}
}

View File

@ -111,7 +111,7 @@ public interface FrameReader extends Closeable {
* @param promisedStreamId server-initiated stream ID. Must be an even
* number.
* @param requestHeaders minimally includes {@code :method}, {@code :scheme},
* {@code :authority}, and (@code :path}.
* {@code :authority}, and {@code :path}.
*/
void pushPromise(int streamId, int promisedStreamId, List<io.grpc.okhttp.internal.framed.Header> requestHeaders)
throws IOException;

View File

@ -45,7 +45,7 @@ public interface FrameWriter extends Closeable {
* @param promisedStreamId server-initiated stream ID. Must be an even
* number.
* @param requestHeaders minimally includes {@code :method}, {@code :scheme},
* {@code :authority}, and (@code :path}.
* {@code :authority}, and {@code :path}.
*/
void pushPromise(int streamId, int promisedStreamId, List<Header> requestHeaders)
throws IOException;

View File

@ -42,7 +42,7 @@ import io.grpc.LoadBalancer.SubchannelPicker;
import io.grpc.LoadBalancerProvider;
import io.grpc.ManagedChannel;
import io.grpc.ManagedChannelBuilder;
import io.grpc.NameResolver.Factory;
import io.grpc.NameResolver;
import io.grpc.Status;
import io.grpc.SynchronizationContext;
import io.grpc.inprocess.InProcessChannelBuilder;
@ -521,7 +521,7 @@ public class CachingRlsLbClientTest {
@Override
@Deprecated
public Factory getNameResolverFactory() {
public NameResolver.Factory getNameResolverFactory() {
throw new UnsupportedOperationException();
}

View File

@ -52,7 +52,7 @@ import io.grpc.MethodDescriptor;
import io.grpc.MethodDescriptor.Marshaller;
import io.grpc.MethodDescriptor.MethodType;
import io.grpc.NameResolver.ConfigOrError;
import io.grpc.NameResolver.Factory;
import io.grpc.NameResolver;
import io.grpc.Status;
import io.grpc.SynchronizationContext;
import io.grpc.inprocess.InProcessChannelBuilder;
@ -509,7 +509,7 @@ public class RlsLoadBalancerTest {
@Override
@Deprecated
public Factory getNameResolverFactory() {
public NameResolver.Factory getNameResolverFactory() {
throw new UnsupportedOperationException();
}

View File

@ -50,7 +50,6 @@ import io.grpc.ServerCallHandler;
import io.grpc.ServerInterceptor;
import io.grpc.Status;
import io.grpc.binarylog.v1.Address;
import io.grpc.binarylog.v1.Address.Type;
import io.grpc.binarylog.v1.GrpcLogEntry;
import io.grpc.binarylog.v1.GrpcLogEntry.EventType;
import io.grpc.binarylog.v1.Message;
@ -791,10 +790,10 @@ final class BinlogHelper {
if (address instanceof InetSocketAddress) {
InetAddress inetAddress = ((InetSocketAddress) address).getAddress();
if (inetAddress instanceof Inet4Address) {
builder.setType(Type.TYPE_IPV4)
builder.setType(Address.Type.TYPE_IPV4)
.setAddress(InetAddressUtil.toAddrString(inetAddress));
} else if (inetAddress instanceof Inet6Address) {
builder.setType(Type.TYPE_IPV6)
builder.setType(Address.Type.TYPE_IPV6)
.setAddress(InetAddressUtil.toAddrString(inetAddress));
} else {
logger.log(Level.SEVERE, "unknown type of InetSocketAddress: {}", address);
@ -803,10 +802,10 @@ final class BinlogHelper {
builder.setIpPort(((InetSocketAddress) address).getPort());
} else if (address.getClass().getName().equals("io.netty.channel.unix.DomainSocketAddress")) {
// To avoid a compile time dependency on grpc-netty, we check against the runtime class name.
builder.setType(Type.TYPE_UNIX)
builder.setType(Address.Type.TYPE_UNIX)
.setAddress(address.toString());
} else {
builder.setType(Type.TYPE_UNKNOWN).setAddress(address.toString());
builder.setType(Address.Type.TYPE_UNKNOWN).setAddress(address.toString());
}
return builder.build();
}

View File

@ -72,7 +72,7 @@ import java.security.cert.CertificateEncodingException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map.Entry;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.logging.Level;
import java.util.logging.Logger;
@ -332,7 +332,7 @@ final class ChannelzProtoUtil {
if (options.tcpInfo != null) {
ret.add(toSocketOptionTcpInfo(options.tcpInfo));
}
for (Entry<String, String> entry : options.others.entrySet()) {
for (Map.Entry<String, String> entry : options.others.entrySet()) {
ret.add(toSocketOptionAdditional(entry.getKey(), entry.getValue()));
}
return ret;

View File

@ -35,7 +35,6 @@ import io.grpc.ClientCall;
import io.grpc.ConnectivityStateInfo;
import io.grpc.LoadBalancer;
import io.grpc.LoadBalancer.CreateSubchannelArgs;
import io.grpc.LoadBalancer.Factory;
import io.grpc.LoadBalancer.Helper;
import io.grpc.LoadBalancer.Subchannel;
import io.grpc.LoadBalancer.SubchannelStateListener;
@ -69,16 +68,16 @@ import javax.annotation.Nullable;
* <p>Note the original LoadBalancer must call {@code Helper.createSubchannel()} from the
* SynchronizationContext, or it will throw.
*/
final class HealthCheckingLoadBalancerFactory extends Factory {
final class HealthCheckingLoadBalancerFactory extends LoadBalancer.Factory {
private static final Logger logger =
Logger.getLogger(HealthCheckingLoadBalancerFactory.class.getName());
private final Factory delegateFactory;
private final LoadBalancer.Factory delegateFactory;
private final BackoffPolicy.Provider backoffPolicyProvider;
private final Supplier<Stopwatch> stopwatchSupplier;
public HealthCheckingLoadBalancerFactory(
Factory delegateFactory, BackoffPolicy.Provider backoffPolicyProvider,
LoadBalancer.Factory delegateFactory, BackoffPolicy.Provider backoffPolicyProvider,
Supplier<Stopwatch> stopwatchSupplier) {
this.delegateFactory = checkNotNull(delegateFactory, "delegateFactory");
this.backoffPolicyProvider = checkNotNull(backoffPolicyProvider, "backoffPolicyProvider");

View File

@ -18,7 +18,6 @@ package io.grpc.services;
import io.grpc.ExperimentalApi;
import io.grpc.LoadBalancer;
import io.grpc.LoadBalancer.Factory;
import io.grpc.LoadBalancer.Helper;
import io.grpc.internal.ExponentialBackoffPolicy;
import io.grpc.internal.GrpcUtil;
@ -35,7 +34,8 @@ public final class HealthCheckingLoadBalancerUtil {
/**
* Creates a health-checking-capable LoadBalancer. This method is used to implement
* health-checking-capable {@link Factory}s, which will typically written this way:
* health-checking-capable {@link io.grpc.LoadBalancer.Factory}s, which will typically written
* this way:
*
* <pre>
* public class HealthCheckingFooLbFactory extends LoadBalancer.Factory {
@ -52,15 +52,15 @@ public final class HealthCheckingLoadBalancerUtil {
* </pre>
*
* <p>As a requirement for the original LoadBalancer, it must call
* {@code Helper.createSubchannel()} from the {@link
* io.grpc.LoadBalancer.Helper#getSynchronizationContext() Synchronization Context}, or
* {@code createSubchannel()} will throw.
* {@code Helper.createSubchannel()} from the {@link Helper#getSynchronizationContext()
* Synchronization Context}, or {@code createSubchannel()} will throw.
*
* @param factory the original factory that implements load-balancing logic without health
* checking
* @param helper the helper passed to the resulting health-checking LoadBalancer.
*/
public static LoadBalancer newHealthCheckingLoadBalancer(Factory factory, Helper helper) {
public static LoadBalancer newHealthCheckingLoadBalancer(
LoadBalancer.Factory factory, Helper helper) {
HealthCheckingLoadBalancerFactory hcFactory =
new HealthCheckingLoadBalancerFactory(
factory, new ExponentialBackoffPolicy.Provider(),

View File

@ -59,7 +59,6 @@ import io.grpc.ServerCallHandler;
import io.grpc.Status;
import io.grpc.StatusException;
import io.grpc.binarylog.v1.Address;
import io.grpc.binarylog.v1.Address.Type;
import io.grpc.binarylog.v1.ClientHeader;
import io.grpc.binarylog.v1.GrpcLogEntry;
import io.grpc.binarylog.v1.GrpcLogEntry.EventType;
@ -377,7 +376,7 @@ public final class BinlogHelperTest {
assertEquals(
Address
.newBuilder()
.setType(Type.TYPE_IPV4)
.setType(Address.Type.TYPE_IPV4)
.setAddress("127.0.0.1")
.setIpPort(12345)
.build(),
@ -393,7 +392,7 @@ public final class BinlogHelperTest {
assertEquals(
Address
.newBuilder()
.setType(Type.TYPE_IPV6)
.setType(Address.Type.TYPE_IPV6)
.setAddress("2001:db8::2:1") // RFC 5952 section 4: ipv6 canonical form required
.setIpPort(12345)
.build(),
@ -407,7 +406,7 @@ public final class BinlogHelperTest {
assertEquals(
Address
.newBuilder()
.setType(Type.TYPE_UNIX)
.setType(Address.Type.TYPE_UNIX)
.setAddress("/some/path")
.build(),
BinlogHelper.socketToProto(socketAddress)
@ -424,7 +423,7 @@ public final class BinlogHelperTest {
};
assertEquals(
Address.newBuilder()
.setType(Type.TYPE_UNKNOWN)
.setType(Address.Type.TYPE_UNKNOWN)
.setAddress("some-socket-address")
.build(),
BinlogHelper.socketToProto(unknownSocket));

View File

@ -85,7 +85,7 @@ import java.net.SocketAddress;
import java.security.cert.Certificate;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map.Entry;
import java.util.Map;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;
@ -932,7 +932,7 @@ public final class ChannelzProtoUtilTest {
SocketOptions.Builder builder = new SocketOptions.Builder()
.setSocketOptionTimeoutMillis(options.soTimeoutMillis)
.setSocketOptionLingerSeconds(options.lingerSeconds);
for (Entry<String, String> entry : options.others.entrySet()) {
for (Map.Entry<String, String> entry : options.others.entrySet()) {
builder.addOption(entry.getKey(), entry.getValue());
}
return builder;

View File

@ -49,7 +49,6 @@ import io.grpc.Context.CancellationListener;
import io.grpc.EquivalentAddressGroup;
import io.grpc.LoadBalancer;
import io.grpc.LoadBalancer.CreateSubchannelArgs;
import io.grpc.LoadBalancer.Factory;
import io.grpc.LoadBalancer.Helper;
import io.grpc.LoadBalancer.ResolvedAddresses;
import io.grpc.LoadBalancer.Subchannel;
@ -131,8 +130,8 @@ public class HealthCheckingLoadBalancerFactoryTest {
private final Helper origHelper = mock(Helper.class, delegatesTo(new FakeHelper()));
// The helper seen by the origLb
private Helper wrappedHelper;
private final Factory origLbFactory =
mock(Factory.class, delegatesTo(new Factory() {
private final LoadBalancer.Factory origLbFactory =
mock(LoadBalancer.Factory.class, delegatesTo(new LoadBalancer.Factory() {
@Override
public LoadBalancer newLoadBalancer(Helper helper) {
checkState(wrappedHelper == null, "LoadBalancer already created");
@ -1056,8 +1055,8 @@ public class HealthCheckingLoadBalancerFactoryTest {
@Test
public void util_newHealthCheckingLoadBalancer() {
Factory hcFactory =
new Factory() {
LoadBalancer.Factory hcFactory =
new LoadBalancer.Factory() {
@Override
public LoadBalancer newLoadBalancer(Helper helper) {
return HealthCheckingLoadBalancerUtil.newHealthCheckingLoadBalancer(

View File

@ -39,6 +39,8 @@ import javax.annotation.concurrent.NotThreadSafe;
* client's perspective by maintaining a set of locality counters for each locality it is tracking
* loads for.
*/
// https://github.com/google/error-prone/issues/1767
@SuppressWarnings("ModifyCollectionInEnhancedForLoop")
@NotThreadSafe
final class LoadStatsStoreImpl implements LoadStatsStore {
private final String clusterName;

View File

@ -30,7 +30,7 @@ import java.util.logging.Logger;
final class MeshCaCertificateProvider extends CertificateProvider {
private static final Logger logger = Logger.getLogger(MeshCaCertificateProvider.class.getName());
protected MeshCaCertificateProvider(DistributorWatcher watcher, boolean notifyCertUpdates,
MeshCaCertificateProvider(DistributorWatcher watcher, boolean notifyCertUpdates,
String meshCaUrl, String zone, long validitySeconds,
int keySize, String alg, String signatureAlg, MeshCaChannelFactory meshCaChannelFactory,
BackoffPolicy.Provider backoffPolicyProvider, long renewalGracePeriodSeconds,

View File

@ -59,7 +59,7 @@ public final class ReferenceCountingMap<K, V extends Closeable> {
/**
* Releases an instance of the given value.
*
* <p>The instance must have been obtained from {@link #get(K)}. Otherwise will throw
* <p>The instance must have been obtained from {@link #get(Object)}. Otherwise will throw
* IllegalArgumentException.
*
* <p>Caller must not release a reference more than once. It's advised that you clear the

View File

@ -67,6 +67,7 @@ public final class StsCredentials extends GoogleCredentials {
this.transportFactory = transportFactory;
}
@SuppressWarnings("JdkObsolete") // We can't use java.time
@Override
public AccessToken refreshAccessToken() throws IOException {
AccessToken tok = getSourceAccessTokenFromFileLocation();

View File

@ -25,7 +25,6 @@ import static org.mockito.Mockito.when;
import com.github.udpa.udpa.data.orca.v1.OrcaLoadReport;
import io.grpc.ClientStreamTracer;
import io.grpc.ClientStreamTracer.Factory;
import io.grpc.ClientStreamTracer.StreamInfo;
import io.grpc.LoadBalancer.PickResult;
import io.grpc.LoadBalancer.PickSubchannelArgs;
@ -224,7 +223,8 @@ public class ClientLoadCounterTest {
}
@Override
protected Factory wrapTracerFactory(Factory originFactory) {
protected ClientStreamTracer.Factory wrapTracerFactory(
ClientStreamTracer.Factory originFactory) {
// NO-OP
return originFactory;
}