MINOR: Remove unnecessary toString(); fix comment references (#13212)

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Divij Vaidya <diviv@amazon.com>, Lucas Brutschy <lbrutschy@confluent.io>
This commit is contained in:
Christo Lolov 2023-03-06 17:39:04 +00:00 committed by GitHub
parent 07e2f6cd4d
commit 5b295293c0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 29 additions and 25 deletions

View File

@ -19,6 +19,8 @@ package org.apache.kafka.clients.admin;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
/** /**
* Options for {@link Admin#alterClientQuotas(Collection, AlterClientQuotasOptions)}. * Options for {@link Admin#alterClientQuotas(Collection, AlterClientQuotasOptions)}.
* *

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.quota.ClientQuotaEntity;
import java.util.Collection;
import java.util.Map; import java.util.Map;
/** /**

View File

@ -20,6 +20,7 @@ package org.apache.kafka.clients.admin;
import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.KafkaFuture;
import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;

View File

@ -1153,7 +1153,7 @@ public abstract class AbstractStickyAssignor extends AbstractPartitionAssignor {
List<String> path = new ArrayList<>(Collections.singleton(pair.srcMemberId)); List<String> path = new ArrayList<>(Collections.singleton(pair.srcMemberId));
if (isLinked(pair.dstMemberId, pair.srcMemberId, reducedPairs, path) && !in(path, cycles)) { if (isLinked(pair.dstMemberId, pair.srcMemberId, reducedPairs, path) && !in(path, cycles)) {
cycles.add(new ArrayList<>(path)); cycles.add(new ArrayList<>(path));
log.error("A cycle of length {} was found: {}", path.size() - 1, path.toString()); log.error("A cycle of length {} was found: {}", path.size() - 1, path);
} }
} }

View File

@ -670,7 +670,7 @@ public class Metrics implements Closeable {
if (!runtimeTagKeys.equals(templateTagKeys)) { if (!runtimeTagKeys.equals(templateTagKeys)) {
throw new IllegalArgumentException("For '" + template.name() + "', runtime-defined metric tags do not match the tags in the template. " throw new IllegalArgumentException("For '" + template.name() + "', runtime-defined metric tags do not match the tags in the template. "
+ "Runtime = " + runtimeTagKeys.toString() + " Template = " + templateTagKeys.toString()); + "Runtime = " + runtimeTagKeys + " Template = " + templateTagKeys.toString());
} }
return this.metricName(template.name(), template.group(), template.description(), tags); return this.metricName(template.name(), template.group(), template.description(), tags);

View File

@ -67,7 +67,7 @@ public interface TransportLayer extends ScatteringByteChannel, TransferableChann
/** /**
* This a no-op for the non-secure PLAINTEXT implementation. For SSL, this performs * This a no-op for the non-secure PLAINTEXT implementation. For SSL, this performs
* SSL handshake. The SSL handshake includes client authentication if configured using * SSL handshake. The SSL handshake includes client authentication if configured using
* {@link org.apache.kafka.common.config.SslConfigs#SSL_CLIENT_AUTH_CONFIG}. * {@link org.apache.kafka.common.config.internals.BrokerSecurityConfigs#SSL_CLIENT_AUTH_CONFIG}.
* @throws AuthenticationException if handshake fails due to an {@link javax.net.ssl.SSLException}. * @throws AuthenticationException if handshake fails due to an {@link javax.net.ssl.SSLException}.
* @throws IOException if read or write fails with an I/O error. * @throws IOException if read or write fails with an I/O error.
*/ */

View File

@ -176,7 +176,7 @@ public class OAuthBearerValidationUtils {
if (!tokenScope.contains(requiredScopeElement)) if (!tokenScope.contains(requiredScopeElement))
return OAuthBearerValidationResult.newFailure(String.format( return OAuthBearerValidationResult.newFailure(String.format(
"The provided scope (%s) was mising a required scope (%s). All required scope elements: %s", "The provided scope (%s) was mising a required scope (%s). All required scope elements: %s",
String.valueOf(tokenScope), requiredScopeElement, requiredScope.toString()), String.valueOf(tokenScope), requiredScopeElement, requiredScope),
requiredScope.toString(), null); requiredScope.toString(), null);
} }
return OAuthBearerValidationResult.newSuccess(); return OAuthBearerValidationResult.newSuccess();

View File

@ -154,7 +154,7 @@ public class ProtocolSerializationTest {
if (!f.def.type.isNullable()) if (!f.def.type.isNullable())
fail("Should not allow serialization of null value."); fail("Should not allow serialization of null value.");
} catch (SchemaException e) { } catch (SchemaException e) {
assertFalse(f.def.type.isNullable(), f.toString() + " should not be nullable"); assertFalse(f.def.type.isNullable(), f + " should not be nullable");
} finally { } finally {
this.struct.set(f, o); this.struct.set(f, o);
} }

View File

@ -54,7 +54,7 @@ public enum EntityType {
} else { } else {
if (!type.toString().equals(baseType.toString())) { if (!type.toString().equals(baseType.toString())) {
throw new RuntimeException("Field " + fieldName + " has entity type " + throw new RuntimeException("Field " + fieldName + " has entity type " +
name() + ", but field type " + type.toString() + ", which does " + name() + ", but field type " + type + ", which does " +
"not match."); "not match.");
} }
} }

View File

@ -454,7 +454,7 @@ public final class FieldSpec {
name + ". The only valid default for a struct field " + name + ". The only valid default for a struct field " +
"is the empty struct or null."); "is the empty struct or null.");
} }
return "new " + type.toString() + "()"; return "new " + type + "()";
} else if (type.isArray()) { } else if (type.isArray()) {
if (fieldDefault.equals("null")) { if (fieldDefault.equals("null")) {
validateNullDefault(); validateNullDefault();

View File

@ -585,7 +585,7 @@ public final class MessageDataGenerator implements MessageClassGenerator {
} else if (type instanceof FieldType.Float64FieldType) { } else if (type instanceof FieldType.Float64FieldType) {
return "_readable.readDouble()"; return "_readable.readDouble()";
} else if (type.isStruct()) { } else if (type.isStruct()) {
return String.format("new %s(_readable, _version)", type.toString()); return String.format("new %s(_readable, _version)", type);
} else { } else {
throw new RuntimeException("Unsupported field type " + type); throw new RuntimeException("Unsupported field type " + type);
} }

View File

@ -317,7 +317,7 @@ final class SchemaGenerator {
fieldTypeToSchemaType(arrayType.elementType(), false, version, fieldFlexibleVersions, false)); fieldTypeToSchemaType(arrayType.elementType(), false, version, fieldFlexibleVersions, false));
} }
} else if (type.isStruct()) { } else if (type.isStruct()) {
return String.format("%s.SCHEMA_%d", type.toString(), return String.format("%s.SCHEMA_%d", type,
floorVersion(type.toString(), version)); floorVersion(type.toString(), version));
} else { } else {
throw new RuntimeException("Unsupported type " + type); throw new RuntimeException("Unsupported type " + type);

View File

@ -594,10 +594,10 @@ public class ReplicationControlManager {
resultsPrefix = ", "; resultsPrefix = ", ";
} }
if (request.validateOnly()) { if (request.validateOnly()) {
log.info("Validate-only CreateTopics result(s): {}", resultsBuilder.toString()); log.info("Validate-only CreateTopics result(s): {}", resultsBuilder);
return ControllerResult.atomicOf(Collections.emptyList(), data); return ControllerResult.atomicOf(Collections.emptyList(), data);
} else { } else {
log.info("CreateTopics result(s): {}", resultsBuilder.toString()); log.info("CreateTopics result(s): {}", resultsBuilder);
return ControllerResult.atomicOf(records, data); return ControllerResult.atomicOf(records, data);
} }
} }
@ -1679,7 +1679,7 @@ public class ReplicationControlManager {
append(record.partitionId()); append(record.partitionId());
prefix = ", "; prefix = ", ";
} }
log.debug("{}: changing partition(s): {}", context, bld.toString()); log.debug("{}: changing partition(s): {}", context, bld);
} else if (log.isInfoEnabled()) { } else if (log.isInfoEnabled()) {
log.info("{}: changing {} partition(s)", context, records.size() - oldSize); log.info("{}: changing {} partition(s)", context, records.size() - oldSize);
} }

View File

@ -121,7 +121,7 @@ public class QuorumState {
// For exceptions during state file loading (missing or not readable), // For exceptions during state file loading (missing or not readable),
// we could assume the file is corrupted already and should be cleaned up. // we could assume the file is corrupted already and should be cleaned up.
log.warn("Clearing local quorum state store after error loading state {}", log.warn("Clearing local quorum state store after error loading state {}",
store.toString(), e); store, e);
store.clear(); store.clear();
election = ElectionState.withUnknownLeader(0, voters); election = ElectionState.withUnknownLeader(0, voters);
} }

View File

@ -69,7 +69,7 @@ public class CheckpointFile<T> {
// Ignore if file already exists. // Ignore if file already exists.
} }
absolutePath = file.toPath().toAbsolutePath(); absolutePath = file.toPath().toAbsolutePath();
tempPath = Paths.get(absolutePath.toString() + ".tmp"); tempPath = Paths.get(absolutePath + ".tmp");
} }
public void write(Collection<T> entries) throws IOException { public void write(Collection<T> entries) throws IOException {

View File

@ -179,7 +179,7 @@ public final class LsCommandHandler implements Commands.Handler {
} }
} }
} }
writer.println(output.toString()); writer.println(output);
} }
} }

View File

@ -149,12 +149,12 @@ public final class MetadataShell {
builder.setSnapshotPath(res.getString("snapshot")); builder.setSnapshotPath(res.getString("snapshot"));
Path tempDir = Files.createTempDirectory("MetadataShell"); Path tempDir = Files.createTempDirectory("MetadataShell");
Exit.addShutdownHook("agent-shutdown-hook", () -> { Exit.addShutdownHook("agent-shutdown-hook", () -> {
log.debug("Removing temporary directory " + tempDir.toAbsolutePath().toString()); log.debug("Removing temporary directory " + tempDir.toAbsolutePath());
try { try {
Utils.delete(tempDir.toFile()); Utils.delete(tempDir.toFile());
} catch (Exception e) { } catch (Exception e) {
log.error("Got exception while removing temporary directory " + log.error("Got exception while removing temporary directory " +
tempDir.toAbsolutePath().toString()); tempDir.toAbsolutePath());
} }
}); });
MetadataShell shell = builder.build(); MetadataShell shell = builder.build();

View File

@ -146,7 +146,7 @@ public class GlobalStateManagerImpl implements GlobalStateManager {
" state store, topic-partition: {}, checkpoint file: {}. If this topic-partition is no longer valid," + " state store, topic-partition: {}, checkpoint file: {}. If this topic-partition is no longer valid," +
" an application reset and state store directory cleanup will be required.", " an application reset and state store directory cleanup will be required.",
tp.topic(), tp.topic(),
checkpointFile.toString() checkpointFile
); );
throw new StreamsException("Encountered a topic-partition not associated with any global state store"); throw new StreamsException("Encountered a topic-partition not associated with any global state store");
} }

View File

@ -1430,7 +1430,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
"%sNumber of assigned partitions %d is not equal to " "%sNumber of assigned partitions %d is not equal to "
+ "the number of active taskIds %d, assignmentInfo=%s", + "the number of active taskIds %d, assignmentInfo=%s",
logPrefix, partitions.size(), logPrefix, partitions.size(),
info.activeTasks().size(), info.toString() info.activeTasks().size(), info
) )
); );
} }

View File

@ -185,7 +185,7 @@ public class CompositeReadOnlySessionStore<K, V> implements ReadOnlySessionStore
throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" + throw new InvalidStateStoreException("State store [" + storeName + "] is not available anymore" +
" and may have been migrated to another instance; " + " and may have been migrated to another instance; " +
"please re-discover its location from the state metadata. " + "please re-discover its location from the state metadata. " +
"Original error message: " + ise.toString()); "Original error message: " + ise);
} }
} }
return KeyValueIterators.emptyIterator(); return KeyValueIterators.emptyIterator();

View File

@ -225,7 +225,7 @@ public class MeteredKeyValueStoreTest {
STORE_LEVEL_GROUP, STORE_LEVEL_GROUP,
THREAD_ID_TAG_KEY, THREAD_ID_TAG_KEY,
threadId, threadId,
taskId.toString(), taskId,
STORE_TYPE, STORE_TYPE,
STORE_NAME STORE_NAME
))); )));

View File

@ -229,7 +229,7 @@ public class MeteredSessionStoreTest {
STORE_LEVEL_GROUP, STORE_LEVEL_GROUP,
THREAD_ID_TAG_KEY, THREAD_ID_TAG_KEY,
threadId, threadId,
taskId.toString(), taskId,
STORE_TYPE, STORE_TYPE,
STORE_NAME STORE_NAME
))); )));

View File

@ -238,7 +238,7 @@ public class MeteredTimestampedKeyValueStoreTest {
STORE_LEVEL_GROUP, STORE_LEVEL_GROUP,
THREAD_ID_TAG_KEY, THREAD_ID_TAG_KEY,
threadId, threadId,
taskId.toString(), taskId,
STORE_TYPE, STORE_TYPE,
STORE_NAME STORE_NAME
))); )));

View File

@ -163,7 +163,7 @@ public class ConsumeBenchWorker implements TaskWorker {
private String consumerGroup() { private String consumerGroup() {
return toUseRandomConsumeGroup() return toUseRandomConsumeGroup()
? "consume-bench-" + UUID.randomUUID().toString() ? "consume-bench-" + UUID.randomUUID()
: spec.consumerGroup(); : spec.consumerGroup();
} }

View File

@ -332,7 +332,7 @@ public class CoordinatorTest {
public ExpectedLines waitFor(final String nodeName, public ExpectedLines waitFor(final String nodeName,
final CapturingCommandRunner runner) throws InterruptedException { final CapturingCommandRunner runner) throws InterruptedException {
TestUtils.waitForCondition(() -> linesMatch(nodeName, runner.lines(nodeName)), TestUtils.waitForCondition(() -> linesMatch(nodeName, runner.lines(nodeName)),
"failed to find the expected lines " + this.toString()); "failed to find the expected lines " + this);
return this; return this;
} }