Skip to content
Open
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
5 changes: 4 additions & 1 deletion build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ plugins {
id 'org.scoverage' version '8.0.3' apply false
id 'com.gradleup.shadow' version '8.3.9' apply false
id 'com.diffplug.spotless' version "7.2.1"
id 'net.ltgt.errorprone' version '4.3.0' apply false
}

ext {
Expand Down Expand Up @@ -76,7 +77,7 @@ ext {
// until version can be upgraded: https://github.com/spotbugs/spotbugs/issues/3564
project.gradle.startParameter.excludedTaskNames.add("spotbugsMain")
project.gradle.startParameter.excludedTaskNames.add("spotbugsTest")
}
}

maxTestForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : Runtime.runtime.availableProcessors()
maxScalacThreads = project.hasProperty('maxScalacThreads') ? maxScalacThreads.toInteger() :
Expand Down Expand Up @@ -228,6 +229,8 @@ allprojects {
delete "${projectDir}/src/generated"
delete "${projectDir}/src/generated-test"
}

apply from: rootProject.file('gradle/error-prone.gradle')
}

def determineCommitId() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -387,7 +387,7 @@ private boolean isSticky() {
if (hasCycles(topicMovementPairs)) {
log.error("Stickiness is violated for topic {}"
+ "\nPartition movements for this topic occurred among the following consumer pairs:"
+ "\n{}", topicMovements.getKey(), topicMovements.getValue().toString());
+ "\n{}", topicMovements.getKey(), topicMovements.getValue());
return false;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,6 @@ public int partition() {

@Override
public String toString() {
return topicPartition.toString() + "@" + offset;
return topicPartition + "@" + offset;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -661,7 +661,7 @@ public MetricName metricInstance(MetricNameTemplate template, Map<String, String

if (!runtimeTagKeys.equals(templateTagKeys)) {
throw new IllegalArgumentException("For '" + template.name() + "', runtime-defined metric tags do not match the tags in the template. "
+ "Runtime = " + runtimeTagKeys + " Template = " + templateTagKeys.toString());
+ "Runtime = " + runtimeTagKeys + " Template = " + templateTagKeys);
}

return this.metricName(template.name(), template.group(), template.description(), tags);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ public void increment(K key) {
synchronized (this) {
if (closed) {
log.warn("{}: Attempted to increment {}, but the GaugeSuite was closed.",
suiteName, key.toString());
suiteName, key);
return;
}
StoredIntGauge gauge = gauges.get(key);
Expand All @@ -181,7 +181,7 @@ public void increment(K key) {
if (gauges.size() == maxEntries) {
if (removable.isEmpty()) {
log.debug("{}: Attempted to increment {}, but there are already {} entries.",
suiteName, key.toString(), maxEntries);
suiteName, key, maxEntries);
return;
}
Iterator<K> iter = removable.iterator();
Expand All @@ -191,13 +191,13 @@ public void increment(K key) {
gauges.remove(keyToRemove);
pending.push(new PendingMetricsChange(metricNameToRemove, null));
log.trace("{}: Removing the metric {}, which has a value of 0.",
suiteName, keyToRemove.toString());
suiteName, keyToRemove);
}
MetricName metricNameToAdd = metricNameCalculator.apply(key);
gauge = new StoredIntGauge(metricNameToAdd);
gauges.put(key, gauge);
pending.push(new PendingMetricsChange(metricNameToAdd, gauge));
log.trace("{}: Adding a new metric {}.", suiteName, key.toString());
log.trace("{}: Adding a new metric {}.", suiteName, key);
}
// Drop the object monitor and perform any pending metrics additions or removals.
performPendingMetricsOperations();
Expand Down Expand Up @@ -236,17 +236,17 @@ private void performPendingMetricsOperations() {
public synchronized void decrement(K key) {
if (closed) {
log.warn("{}: Attempted to decrement {}, but the gauge suite was closed.",
suiteName, key.toString());
suiteName, key);
return;
}
StoredIntGauge gauge = gauges.get(key);
if (gauge == null) {
log.debug("{}: Attempted to decrement {}, but no such metric was registered.",
suiteName, key.toString());
suiteName, key);
} else {
int cur = gauge.decrement();
log.trace("{}: Removed a reference to {}. {} reference(s) remaining.",
suiteName, key.toString(), cur);
suiteName, key, cur);
if (cur <= 0) {
removable.add(key);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ public static String deepToString(Iterator<?> iter) {
while (iter.hasNext()) {
Object object = iter.next();
bld.append(prefix);
bld.append(object.toString());
bld.append(object);
prefix = ", ";
}
bld.append("]");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ public String toString() {
StringBuilder b = new StringBuilder();
b.append('{');
for (int i = 0; i < this.fields.length; i++) {
b.append(this.fields[i].toString());
b.append(this.fields[i]);
if (i < this.fields.length - 1)
b.append(',');
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ public String toString() {
for (Map.Entry<Integer, Field> field : fields.entrySet()) {
bld.append(prefix);
prefix = ", ";
bld.append(field.getKey()).append(" -> ").append(field.getValue().toString());
bld.append(field.getKey()).append(" -> ").append(field.getValue());
}
bld.append(")");
return bld.toString();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -892,7 +892,7 @@ private KafkaMetric getMetric(String name, Map<String, String> tags) throws Exce
entry.getKey().name().equals(name) && entry.getKey().tags().equals(tags))
.findFirst();
if (metric.isEmpty())
throw new Exception(String.format("Could not find metric called %s with tags %s", name, tags.toString()));
throw new Exception(String.format("Could not find metric called %s with tags %s", name, tags));

return metric.get().getValue();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -226,12 +226,12 @@ public void awaitStop() {

private void checkHerder(SourceAndTarget sourceAndTarget) {
if (!herders.containsKey(sourceAndTarget)) {
throw new IllegalArgumentException("No herder for " + sourceAndTarget.toString());
throw new IllegalArgumentException("No herder for " + sourceAndTarget);
}
}

private void addHerder(SourceAndTarget sourceAndTarget) {
log.info("creating herder for {}", sourceAndTarget.toString());
log.info("creating herder for {}", sourceAndTarget);
Map<String, String> workerProps = config.workerConfig(sourceAndTarget);
DistributedConfig distributedConfig = new DistributedConfig(workerProps);
String encodedSource = encodePath(sourceAndTarget.source());
Expand Down Expand Up @@ -289,13 +289,13 @@ private static String encodePath(String rawPath) {

private String generateWorkerId(SourceAndTarget sourceAndTarget) {
if (config.enableInternalRest()) {
return advertisedUrl.getHost() + ":" + advertisedUrl.getPort() + "/" + sourceAndTarget.toString();
return advertisedUrl.getHost() + ":" + advertisedUrl.getPort() + "/" + sourceAndTarget;
}
try {
//UUID to make sure it is unique even if multiple workers running on the same host
return InetAddress.getLocalHost().getCanonicalHostName() + "/" + sourceAndTarget.toString() + "/" + UUID.randomUUID();
return InetAddress.getLocalHost().getCanonicalHostName() + "/" + sourceAndTarget + "/" + UUID.randomUUID();
} catch (UnknownHostException e) {
return sourceAndTarget.toString() + "/" + UUID.randomUUID();
return sourceAndTarget + "/" + UUID.randomUUID();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public void configure(Map<String, ?> configs, boolean isKey) {
@Override
public byte[] fromConnectData(String topic, Schema schema, Object value) {
if (schema != null && schema.type() != Type.BOOLEAN)
throw new DataException("Invalid schema type for BooleanConverter: " + schema.type().toString());
throw new DataException("Invalid schema type for BooleanConverter: " + schema.type());

try {
return serializer.serialize(topic, (Boolean) value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public void configure(Map<String, ?> configs, boolean isKey) {
@Override
public byte[] fromConnectData(String topic, Schema schema, Object value) {
if (schema != null && schema.type() != Schema.Type.BYTES)
throw new DataException("Invalid schema type for ByteArrayConverter: " + schema.type().toString());
throw new DataException("Invalid schema type for ByteArrayConverter: " + schema.type());

if (value != null && !(value instanceof byte[]) && !(value instanceof ByteBuffer))
throw new DataException("ByteArrayConverter is not compatible with objects of type " + value.getClass());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ static String prefixFor(String connectorName, Scope scope, Integer taskNumber) {
// Append non-task scopes (e.g., worker and offset)
if (scope != Scope.TASK) {
sb.append("|");
sb.append(scope.toString());
sb.append(scope);
}
sb.append("] ");
return sb.toString();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -283,7 +283,7 @@ private static Object castValueToType(Schema schema, Object value, Schema.Type t
default -> throw new DataException(targetType + " is not supported in the Cast transformation.");
};
} catch (NumberFormatException e) {
throw new DataException("Value (" + value.toString() + ") was out of range for requested data type", e);
throw new DataException("Value (" + value + ") was out of range for requested data type", e);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -379,7 +379,7 @@ public String elementName() {

@Override
public String toString() {
return "[]" + elementType.toString();
return "[]" + elementType;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -262,7 +262,7 @@ private void generateVariableLengthTargetFromJson(Target target, Versions curVer
} else if (target.field().type().isStruct()) {
buffer.printf("%s;%n", target.assignmentStatement(
String.format("%s%s.read(%s, _version)",
target.field().type().toString(), SUFFIX, target.sourceVariable())));
target.field().type(), SUFFIX, target.sourceVariable())));
} else {
throw new RuntimeException("Unexpected type " + target.field().type());
}
Expand Down Expand Up @@ -439,7 +439,7 @@ private void generateVariableLengthTargetToJson(Target target, Versions versions
} else if (target.field().type().isStruct()) {
buffer.printf("%s;%n", target.assignmentStatement(
String.format("%sJsonConverter.write(%s, _version, _serializeRecords)",
target.field().type().toString(), target.sourceVariable())));
target.field().type(), target.sourceVariable())));
} else {
throw new RuntimeException("unknown type " + target.field().type());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ public static void processDirectories(String packageName,
numProcessed++;
typeClassGenerators.forEach(generator -> generator.registerMessageType(spec));
} catch (Exception e) {
throw new RuntimeException("Exception while processing " + inputPath.toString(), e);
throw new RuntimeException("Exception while processing " + inputPath, e);
}
}
}
Expand Down
37 changes: 37 additions & 0 deletions gradle/error-prone.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
import static java.lang.System.getenv

// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

apply plugin: 'net.ltgt.errorprone'

dependencies {
errorprone('com.google.errorprone:error_prone_core:2.42.0')
errorprone('tech.picnic.error-prone-support:error-prone-contrib:0.25.0')
}

tasks.withType(JavaCompile).configureEach {
options.errorprone {
disableAllChecks = true // consider removal to avoid error prone error´s, following convention over configuration.
error('RedundantStringConversion')
errorproneArgs.add('-XepExcludedPaths:.*/build/generated/.*')
if (!getenv().containsKey('CI') && getenv('IN_PLACE')?.toBoolean()) {
errorproneArgs.addAll(
'-XepPatchLocation:IN_PLACE',
'-XepPatchChecks:RedundantStringConversion'
)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -3699,8 +3699,8 @@ private StreamsGroupMember maybeReconcile(
log.info("[GroupId {}][MemberId {}] Member's new assignment state: epoch={}, previousEpoch={}, state={}, "
+ "assignedTasks={} and tasksPendingRevocation={}.",
groupId, updatedMember.memberId(), updatedMember.memberEpoch(), updatedMember.previousMemberEpoch(), updatedMember.state(),
updatedMember.assignedTasks().toString(),
updatedMember.tasksPendingRevocation().toString());
updatedMember.assignedTasks(),
updatedMember.tasksPendingRevocation());

// Schedule/cancel the rebalance timeout.
if (updatedMember.state() == org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -659,10 +659,10 @@ private void replayRegistrationChange(
BrokerRegistration curRegistration = brokerRegistrations.get(brokerId);
if (curRegistration == null) {
throw new RuntimeException(String.format("Unable to replay %s: no broker " +
"registration found for that id", record.toString()));
"registration found for that id", record));
} else if (curRegistration.epoch() != brokerEpoch) {
throw new RuntimeException(String.format("Unable to replay %s: no broker " +
"registration with that epoch found", record.toString()));
"registration with that epoch found", record));
} else {
BrokerRegistration nextRegistration = curRegistration.cloneWith(
fencingChange,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ private void maybePublishMetadata(MetadataDelta delta, MetadataImage image, Load
this.image = image;

if (stillNeedToCatchUp(
"maybePublishMetadata(" + manifest.type().toString() + ")",
"maybePublishMetadata(" + manifest.type() + ")",
manifest.provenance().lastContainedOffset())
) {
return;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,9 @@ public String toString() {
if (min == max) {
return String.valueOf(min);
} else if (max == Short.MAX_VALUE) {
return String.valueOf(min) + "+";
return min + "+";
} else {
return String.valueOf(min) + "-" + String.valueOf(max);
return min + "-" + max;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ public synchronized void handleCommit(BatchReader<ApiMessageAndVersion> reader)

for (ApiMessageAndVersion messageAndVersion : batch.records()) {
ApiMessage message = messageAndVersion.message();
serializedEvents.add(COMMIT + " " + message.toString());
serializedEvents.add(COMMIT + " " + message);
}
serializedEvents.add(LAST_COMMITTED_OFFSET + " " + lastCommittedOffset);
}
Expand All @@ -72,7 +72,7 @@ public synchronized void handleLoadSnapshot(SnapshotReader<ApiMessageAndVersion>

for (ApiMessageAndVersion messageAndVersion : batch.records()) {
ApiMessage message = messageAndVersion.message();
serializedEvents.add(SNAPSHOT + " " + message.toString());
serializedEvents.add(SNAPSHOT + " " + message);
}
serializedEvents.add(LAST_COMMITTED_OFFSET + " " + lastCommittedOffset);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public String toString() {
for (DynamicVoter voter : voters.values()) {
builder.append(prefix);
prefix = ",";
builder.append(voter.toString());
builder.append(voter);
}
return builder.toString();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -478,7 +478,7 @@ public void testInvalidVoterReplicaPreVoteRequest(KRaftVersion kraftVersion) thr
// invalid voter id is rejected
context.deliverRequest(
context.voteRequest(
context.clusterId.toString(),
context.clusterId,
epoch,
otherNodeKey,
ReplicaKey.of(10, Uuid.randomUuid()),
Expand All @@ -493,7 +493,7 @@ public void testInvalidVoterReplicaPreVoteRequest(KRaftVersion kraftVersion) thr
// invalid voter directory id is rejected
context.deliverRequest(
context.voteRequest(
context.clusterId.toString(),
context.clusterId,
epoch,
otherNodeKey,
ReplicaKey.of(0, Uuid.randomUuid()),
Expand Down
Loading