kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ij...@apache.org
Subject kafka git commit: KAFKA-4500; Code quality improvements
Date Tue, 20 Dec 2016 12:40:51 GMT
Repository: kafka
Updated Branches:
  refs/heads/trunk 135860225 -> a5c15ba03


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


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a5c15ba0
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a5c15ba0
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a5c15ba0

Branch: refs/heads/trunk
Commit: a5c15ba0374acbb43f69b1a556077e4ad53332e8
Parents: 1358602
Author: Rekha Joshi <rekhajoshm@gmail.com>
Authored: Tue Dec 20 12:40:00 2016 +0000
Committer: Ismael Juma <ismael@juma.me.uk>
Committed: Tue Dec 20 12:40:07 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/kafka/common/config/ConfigValue.java  |  2 +-
 .../apache/kafka/common/metrics/stats/Percentiles.java    |  2 +-
 .../main/java/org/apache/kafka/common/network/Mode.java   |  2 +-
 .../java/org/apache/kafka/common/protocol/Errors.java     |  2 +-
 .../apache/kafka/common/protocol/SecurityProtocol.java    |  2 +-
 .../kafka/common/record/KafkaLZ4BlockInputStream.java     |  3 +--
 .../kafka/common/requests/ControlledShutdownRequest.java  |  4 ++--
 .../producer/internals/ProducerInterceptorsTest.java      |  3 +--
 .../org/apache/kafka/test/MockProducerInterceptor.java    |  3 +--
 .../apache/kafka/connect/storage/OffsetBackingStore.java  | 10 +++++-----
 .../connect/runtime/standalone/StandaloneHerderTest.java  |  2 +-
 core/src/main/scala/kafka/consumer/SimpleConsumer.scala   |  2 +-
 .../apache/kafka/log4jappender/KafkaLog4jAppender.java    |  4 +---
 .../java/org/apache/kafka/tools/ThroughputThrottler.java  |  9 +++++----
 .../java/org/apache/kafka/tools/VerifiableConsumer.java   |  4 ++--
 .../org/apache/kafka/tools/VerifiableLog4jAppender.java   |  7 +------
 .../java/org/apache/kafka/tools/VerifiableProducer.java   |  7 +------
 17 files changed, 27 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java
index 985e05f..a62fcc2 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java
@@ -96,7 +96,7 @@ public class ConfigValue {
 
     @Override
     public String toString() {
-        StringBuffer sb = new StringBuffer();
+        StringBuilder sb = new StringBuilder();
         sb.append("[")
             .append(name)
             .append(",")

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
index 78c93e8..12064f5 100644
--- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
@@ -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
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/network/Mode.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Mode.java b/clients/src/main/java/org/apache/kafka/common/network/Mode.java
index 67de44d..4d8ef3b 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Mode.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Mode.java
@@ -16,4 +16,4 @@
  */
 package org.apache.kafka.common.network;
 
-public enum Mode { CLIENT, SERVER };
+public enum Mode { CLIENT, SERVER }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index f32399d..3905c82 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -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;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
index d5fbed7..fe48832 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java
@@ -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;

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java
b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java
index 92718d8..a408580 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java
@@ -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

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
index c2ace32..24adb36 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ControlledShutdownRequest.java
@@ -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:

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
index 2135eb2..659a29f 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
@@ -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

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java
index 9c4721b..22a588d 100644
--- a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java
+++ b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java
@@ -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

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java
b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java
index 1b74a90..e307eed 100644
--- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java
+++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetBackingStore.java
@@ -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);
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
index 010c0b2..e05aa41 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java
@@ -84,7 +84,7 @@ public class StandaloneHerderTest {
 
     private enum SourceSink {
         SOURCE, SINK
-    };
+    }
 
     private StandaloneHerder herder;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
index e315678..5d219ff 100644
--- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
+++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
@@ -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._
 
 /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
----------------------------------------------------------------------
diff --git a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
index 5759105..ed96e7b 100644
--- a/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
+++ b/log4j-appender/src/main/java/org/apache/kafka/log4jappender/KafkaLog4jAppender.java
@@ -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);
             }
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java b/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java
index 68de638..b550484 100644
--- a/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java
+++ b/tools/src/main/java/org/apache/kafka/tools/ThroughputThrottler.java
@@ -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;
 
     /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java
index 8db442e..cd17217 100644
--- a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java
+++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java
@@ -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);

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java
index ffbf7dc..daf569c 100644
--- a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java
+++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java
@@ -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;
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/a5c15ba0/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
index 30f08e8..54d53f1 100644
--- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
+++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java
@@ -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;
     }


Mime
View raw message