KAFKA-4500; Code quality improvements

- Removed redundant modifiers, not needed String.format()
- Removed unnecessary semicolon, additional assignment, inlined return
- Using StringBuilder for consistency across codebase
- Using try-with-resources

Author: Rekha Joshi <rekhajoshm@gmail.com>
Author: Joshi <rekhajoshm@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #2222 from rekhajoshm/KAFKA-4500
This commit is contained in:
Rekha Joshi 2016-12-20 12:40:00 +00:00 committed by Ismael Juma
parent 135860225e
commit a5c15ba037
17 changed files with 27 additions and 41 deletions

View File

@ -96,7 +96,7 @@ public class ConfigValue {
@Override
public String toString() {
StringBuffer sb = new StringBuffer();
StringBuilder sb = new StringBuilder();
sb.append("[")
.append(name)
.append(",")

View File

@ -27,7 +27,7 @@ import org.apache.kafka.common.metrics.stats.Histogram.LinearBinScheme;
*/
public class Percentiles extends SampledStat implements CompoundStat {
public static enum BucketSizing {
public enum BucketSizing {
CONSTANT, LINEAR
}

View File

@ -16,4 +16,4 @@
*/
package org.apache.kafka.common.network;
public enum Mode { CLIENT, SERVER };
public enum Mode { CLIENT, SERVER }

View File

@ -182,7 +182,7 @@ public enum Errors {
private final short code;
private final ApiException exception;
private Errors(int code, ApiException exception) {
Errors(int code, ApiException exception) {
this.code = (short) code;
this.exception = exception;
}

View File

@ -66,7 +66,7 @@ public enum SecurityProtocol {
/* Whether this security protocol is for testing/debugging */
private final boolean isTesting;
private SecurityProtocol(int id, String name, boolean isTesting) {
SecurityProtocol(int id, String name, boolean isTesting) {
this.id = (short) id;
this.name = name;
this.isTesting = isTesting;

View File

@ -199,9 +199,8 @@ public final class KafkaLZ4BlockInputStream extends FilterInputStream {
if (finished) {
return -1;
}
int value = buffer[bufferOffset++] & 0xFF;
return value;
return buffer[bufferOffset++] & 0xFF;
}
@Override

View File

@ -45,8 +45,8 @@ public class ControlledShutdownRequest extends AbstractRequest {
public AbstractResponse getErrorResponse(int versionId, Throwable e) {
switch (versionId) {
case 0:
throw new IllegalArgumentException(String.format("Version 0 is not supported. It is only supported by " +
"the Scala request class for controlled shutdown"));
throw new IllegalArgumentException("Version 0 is not supported. It is only supported by " +
"the Scala request class for controlled shutdown");
case 1:
return new ControlledShutdownResponse(Errors.forException(e).code(), Collections.<TopicPartition>emptySet());
default:

View File

@ -54,9 +54,8 @@ public class ProducerInterceptorsTest {
if (throwExceptionOnSend)
throw new KafkaException("Injected exception in AppendProducerInterceptor.onSend");
ProducerRecord<Integer, String> newRecord = new ProducerRecord<>(
return new ProducerRecord<>(
record.topic(), record.partition(), record.key(), record.value().concat(appendStr));
return newRecord;
}
@Override

View File

@ -63,9 +63,8 @@ public class MockProducerInterceptor implements ClusterResourceListener, Produce
@Override
public ProducerRecord<String, String> onSend(ProducerRecord<String, String> record) {
ONSEND_COUNT.incrementAndGet();
ProducerRecord<String, String> newRecord = new ProducerRecord<>(
return new ProducerRecord<>(
record.topic(), record.partition(), record.key(), record.value().concat(appendStr));
return newRecord;
}
@Override

View File

@ -43,13 +43,13 @@ public interface OffsetBackingStore {
/**
* Start this offset store.
*/
public void start();
void start();
/**
* Stop the backing store. Implementations should attempt to shutdown gracefully, but not block
* indefinitely.
*/
public void stop();
void stop();
/**
* Get the values for the specified keys
@ -57,7 +57,7 @@ public interface OffsetBackingStore {
* @param callback callback to invoke on completion
* @return future for the resulting map from key to value
*/
public Future<Map<ByteBuffer, ByteBuffer>> get(
Future<Map<ByteBuffer, ByteBuffer>> get(
Collection<ByteBuffer> keys,
Callback<Map<ByteBuffer, ByteBuffer>> callback);
@ -67,12 +67,12 @@ public interface OffsetBackingStore {
* @param callback callback to invoke on completion
* @return void future for the operation
*/
public Future<Void> set(Map<ByteBuffer, ByteBuffer> values,
Future<Void> set(Map<ByteBuffer, ByteBuffer> values,
Callback<Void> callback);
/**
* Configure class with the given key-value pairs
* @param config can be DistributedConfig or StandaloneConfig
*/
public void configure(WorkerConfig config);
void configure(WorkerConfig config);
}

View File

@ -84,7 +84,7 @@ public class StandaloneHerderTest {
private enum SourceSink {
SOURCE, SINK
};
}
private StandaloneHerder herder;

View File

@ -25,7 +25,7 @@ import kafka.api._
import kafka.network._
import kafka.utils._
import kafka.common.{ErrorMapping, TopicAndPartition}
import org.apache.kafka.common.network.{NetworkReceive, Receive}
import org.apache.kafka.common.network.{NetworkReceive}
import org.apache.kafka.common.utils.Utils._
/**

View File

@ -254,9 +254,7 @@ public class KafkaLog4jAppender extends AppenderSkeleton {
if (syncSend) {
try {
response.get();
} catch (InterruptedException ex) {
throw new RuntimeException(ex);
} catch (ExecutionException ex) {
} catch (InterruptedException | ExecutionException ex) {
throw new RuntimeException(ex);
}
}

View File

@ -42,10 +42,11 @@ public class ThroughputThrottler {
private static final long NS_PER_SEC = 1000 * NS_PER_MS;
private static final long MIN_SLEEP_NS = 2 * NS_PER_MS;
long sleepTimeNs;
long sleepDeficitNs = 0;
long targetThroughput = -1;
long startMs;
private final long startMs;
private final long sleepTimeNs;
private final long targetThroughput;
private long sleepDeficitNs = 0;
private boolean wakeup = false;
/**

View File

@ -46,8 +46,8 @@ import java.io.Closeable;
import java.io.IOException;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -215,7 +215,7 @@ public class VerifiableConsumer implements Closeable, OffsetCommitCallback, Cons
public void run() {
try {
consumer.subscribe(Arrays.asList(topic), this);
consumer.subscribe(Collections.singletonList(topic), this);
while (true) {
ConsumerRecords<String, String> records = consumer.poll(Long.MAX_VALUE);

View File

@ -163,13 +163,8 @@ public class VerifiableLog4jAppender {
*/
public static Properties loadProps(String filename) throws IOException, FileNotFoundException {
Properties props = new Properties();
InputStream propStream = null;
try {
propStream = new FileInputStream(filename);
try (InputStream propStream = new FileInputStream(filename)) {
props.load(propStream);
} finally {
if (propStream != null)
propStream.close();
}
return props;
}

View File

@ -165,13 +165,8 @@ public class VerifiableProducer {
*/
public static Properties loadProps(String filename) throws IOException, FileNotFoundException {
Properties props = new Properties();
InputStream propStream = null;
try {
propStream = new FileInputStream(filename);
try (InputStream propStream = new FileInputStream(filename)) {
props.load(propStream);
} finally {
if (propStream != null)
propStream.close();
}
return props;
}