Skip to content

Commit

Permalink
Run sort imports + redundant braces on 1 line statements
Browse files Browse the repository at this point in the history
  • Loading branch information
olafurpg committed Sep 25, 2016
1 parent efe7f84 commit 3c82a0c
Show file tree
Hide file tree
Showing 182 changed files with 425 additions and 602 deletions.
5 changes: 5 additions & 0 deletions .scalafmt.conf
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
rewrite.rules = [
SortImports,
RedundantBraces
]
rewrite.redundantBraces.maxLines = 1
4 changes: 1 addition & 3 deletions repos/kafka/core/src/main/scala/kafka/Kafka.scala
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,7 @@ object Kafka extends Logging {
Runtime
.getRuntime()
.addShutdownHook(new Thread() {
override def run() = {
kafkaServerStartable.shutdown
}
override def run() = kafkaServerStartable.shutdown
})

kafkaServerStartable.startup
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,9 +131,8 @@ class AdminClient(val time: Time,
}
}

def listAllGroupsFlattened(): List[GroupOverview] = {
def listAllGroupsFlattened(): List[GroupOverview] =
listAllGroups.values.flatten.toList
}

def listAllConsumerGroupsFlattened(): List[GroupOverview] = {
listAllGroupsFlattened.filter(
Expand Down
7 changes: 3 additions & 4 deletions repos/kafka/core/src/main/scala/kafka/admin/AdminUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@ import java.util.Random
import java.util.Properties
import org.apache.kafka.common.Node
import org.apache.kafka.common.errors.{
ReplicaNotAvailableException,
InvalidTopicException,
LeaderNotAvailableException
LeaderNotAvailableException,
ReplicaNotAvailableException
}
import org.apache.kafka.common.protocol.{Errors, SecurityProtocol}
import org.apache.kafka.common.requests.MetadataResponse
Expand Down Expand Up @@ -390,9 +390,8 @@ object AdminUtils extends Logging {
}
}

def isConsumerGroupActive(zkUtils: ZkUtils, group: String) = {
def isConsumerGroupActive(zkUtils: ZkUtils, group: String) =
zkUtils.getConsumersInGroup(group).nonEmpty
}

/**
* Delete the whole directory of the given consumer group if the group is inactive.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.kafka.common.serialization.StringDeserializer
import org.apache.kafka.common.utils.Utils

import scala.collection.JavaConverters._
import scala.collection.{Set, mutable}
import scala.collection.{mutable, Set}

object ConsumerGroupCommand {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import joptsimple.OptionParser
import kafka.utils._
import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import kafka.common.{TopicAndPartition, AdminCommandFailedException}
import kafka.common.{AdminCommandFailedException, TopicAndPartition}
import collection._
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.security.JaasUtils
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import kafka.utils._
import collection._
import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import kafka.common.{TopicAndPartition, AdminCommandFailedException}
import kafka.common.{AdminCommandFailedException, TopicAndPartition}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.security.JaasUtils

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.util.concurrent.ThreadPoolExecutor
import java.util.concurrent.TimeUnit
import joptsimple.OptionParser
import org.I0Itec.zkclient.exception.ZkException
import kafka.utils.{Logging, ZkUtils, CommandLineUtils}
import kafka.utils.{CommandLineUtils, Logging, ZkUtils}
import org.apache.log4j.Level
import org.apache.kafka.common.security.JaasUtils
import org.apache.zookeeper.AsyncCallback.{ChildrenCallback, StatCallback}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import java.nio.ByteBuffer

import kafka.common.{TopicAndPartition}
import kafka.api.ApiUtils._
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
Expand Down Expand Up @@ -63,9 +63,7 @@ case class ControlledShutdownRequest(versionId: Short,
4 /* broker id */
}

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def handleError(e: Throwable,
requestChannel: RequestChannel,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,5 @@ case class ControlledShutdownResponse(
}
}

override def describe(details: Boolean): String = { toString }

override def describe(details: Boolean): String = toString
}
4 changes: 1 addition & 3 deletions repos/kafka/core/src/main/scala/kafka/api/FetchRequest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -149,9 +149,7 @@ case class FetchRequest(

def numPartitions = requestInfo.size

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def handleError(e: Throwable,
requestChannel: RequestChannel,
Expand Down
4 changes: 2 additions & 2 deletions repos/kafka/core/src/main/scala/kafka/api/FetchResponse.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@ import java.nio.ByteBuffer
import java.nio.channels.GatheringByteChannel

import kafka.common.TopicAndPartition
import kafka.message.{MessageSet, ByteBufferMessageSet}
import kafka.message.{ByteBufferMessageSet, MessageSet}
import kafka.api.ApiUtils._
import org.apache.kafka.common.KafkaException
import org.apache.kafka.common.network.{Send, MultiSend}
import org.apache.kafka.common.network.{MultiSend, Send}
import org.apache.kafka.common.protocol.Errors

import scala.collection._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,7 @@ private[kafka] abstract class GenericRequestAndHeader(
body.sizeOf()
}

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def describe(details: Boolean): String = {
val strBuffer = new StringBuilder
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,7 @@ private[kafka] abstract class GenericResponseAndHeader(
body.sizeOf()
}

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def describe(details: Boolean): String = {
val strBuffer = new StringBuilder
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package kafka.api

import java.nio.ByteBuffer

import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import org.apache.kafka.common.protocol.{ApiKeys, Errors}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import java.nio.ByteBuffer

import kafka.api.ApiUtils._
import kafka.common.{OffsetAndMetadata, TopicAndPartition}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
Expand Down Expand Up @@ -204,7 +204,5 @@ case class OffsetCommitRequest(
offsetCommitRequest.toString()
}

override def toString = {
describe(details = true)
}
override def toString = describe(details = true)
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,5 @@ case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short],
partitionStatus.size * (4 /* partition */ + 2 /* error code */ )
})

override def describe(details: Boolean): String = { toString }

override def describe(details: Boolean): String = toString
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import java.nio.ByteBuffer

import kafka.api.ApiUtils._
import kafka.common.{TopicAndPartition, _}
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
Expand Down Expand Up @@ -125,7 +125,5 @@ case class OffsetFetchRequest(
offsetFetchRequest.toString()
}

override def toString: String = {
describe(details = true)
}
override def toString: String = describe(details = true)
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package kafka.api
import java.nio.ByteBuffer

import kafka.api.ApiUtils._
import kafka.common.{TopicAndPartition, OffsetMetadataAndError}
import kafka.common.{OffsetMetadataAndError, TopicAndPartition}
import kafka.utils.Logging

object OffsetFetchResponse extends Logging {
Expand Down Expand Up @@ -84,5 +84,5 @@ case class OffsetFetchResponse(
})
})

override def describe(details: Boolean): String = { toString }
override def describe(details: Boolean): String = toString
}
6 changes: 2 additions & 4 deletions repos/kafka/core/src/main/scala/kafka/api/OffsetRequest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import java.nio.ByteBuffer

import kafka.api.ApiUtils._
import kafka.common.TopicAndPartition
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import org.apache.kafka.common.protocol.{ApiKeys, Errors}

Expand Down Expand Up @@ -120,9 +120,7 @@ case class OffsetRequest(
def isFromOrdinaryClient = replicaId == Request.OrdinaryConsumerId
def isFromDebuggingClient = replicaId == Request.DebuggingConsumerId

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def handleError(e: Throwable,
requestChannel: RequestChannel,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,5 +97,5 @@ case class OffsetResponse(
}
}

override def describe(details: Boolean): String = { toString }
override def describe(details: Boolean): String = toString
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.nio._
import kafka.api.ApiUtils._
import kafka.common._
import kafka.message._
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import org.apache.kafka.common.protocol.{ApiKeys, Errors}

Expand Down Expand Up @@ -136,9 +136,7 @@ case class ProducerRequest(

def numPartitions = data.size

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def handleError(e: Throwable,
requestChannel: RequestChannel,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,5 +110,5 @@ case class ProducerResponse(
buffer.putInt(throttleTime)
}

override def describe(details: Boolean): String = { toString }
override def describe(details: Boolean): String = toString
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package kafka.api
import java.nio.ByteBuffer

import kafka.api.ApiUtils._
import kafka.network.{RequestOrResponseSend, RequestChannel}
import kafka.network.{RequestChannel, RequestOrResponseSend}
import kafka.network.RequestChannel.Response
import kafka.utils.Logging
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
Expand Down Expand Up @@ -59,9 +59,7 @@ case class TopicMetadataRequest(versionId: Short,
topics.foldLeft(0)(_ + shortStringLength(_)) /* topics */
}

override def toString(): String = {
describe(true)
}
override def toString(): String = describe(true)

override def handleError(e: Throwable,
requestChannel: RequestChannel,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,5 +54,5 @@ case class TopicMetadataResponse(brokers: Seq[BrokerEndPoint],
topicsMetadata.foreach(_.writeTo(buffer))
}

override def describe(details: Boolean): String = { toString }
override def describe(details: Boolean): String = toString
}
19 changes: 5 additions & 14 deletions repos/kafka/core/src/main/scala/kafka/cluster/Partition.scala
Original file line number Diff line number Diff line change
Expand Up @@ -74,9 +74,7 @@ class Partition(val topic: String,
val tags = Map("topic" -> topic, "partition" -> partitionId.toString)

newGauge("UnderReplicated", new Gauge[Int] {
def value = {
if (isUnderReplicated) 1 else 0
}
def value = if (isUnderReplicated) 1 else 0
}, tags)

def isUnderReplicated(): Boolean = {
Expand Down Expand Up @@ -139,13 +137,10 @@ class Partition(val topic: String,
}
}

def addReplicaIfNotExists(replica: Replica) = {
def addReplicaIfNotExists(replica: Replica) =
assignedReplicaMap.putIfNotExists(replica.brokerId, replica)
}

def assignedReplicas(): Set[Replica] = {
assignedReplicaMap.values.toSet
}
def assignedReplicas(): Set[Replica] = assignedReplicaMap.values.toSet

def removeReplica(replicaId: Int) {
assignedReplicaMap.remove(replicaId)
Expand All @@ -170,9 +165,7 @@ class Partition(val topic: String,
}
}

def getLeaderEpoch(): Int = {
return this.leaderEpoch
}
def getLeaderEpoch(): Int = return this.leaderEpoch

/**
* Make the local replica the leader by resetting LogEndOffset for remote replicas (there could be old LogEndOffset
Expand Down Expand Up @@ -557,9 +550,7 @@ class Partition(val topic: String,
false
}

override def hashCode(): Int = {
31 + topic.hashCode() + 17 * partitionId
}
override def hashCode(): Int = 31 + topic.hashCode() + 17 * partitionId

override def toString(): String = {
val partitionString = new StringBuilder
Expand Down
7 changes: 3 additions & 4 deletions repos/kafka/core/src/main/scala/kafka/cluster/Replica.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@
package kafka.cluster

import kafka.log.Log
import kafka.utils.{SystemTime, Time, Logging}
import kafka.server.{LogReadResult, LogOffsetMetadata}
import kafka.utils.{Logging, SystemTime, Time}
import kafka.server.{LogOffsetMetadata, LogReadResult}
import kafka.common.KafkaException

import java.util.concurrent.atomic.AtomicLong
Expand Down Expand Up @@ -119,9 +119,8 @@ class Replica(val brokerId: Int,
false
}

override def hashCode(): Int = {
override def hashCode(): Int =
31 + topic.hashCode() + 17 * brokerId + partition.hashCode()
}

override def toString(): String = {
val replicaString = new StringBuilder
Expand Down
8 changes: 2 additions & 6 deletions repos/kafka/core/src/main/scala/kafka/common/AppInfo.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,15 +32,11 @@ object AppInfo extends KafkaMetricsGroup {
}

newGauge("Version", new Gauge[String] {
def value = {
AppInfoParser.getVersion()
}
def value = AppInfoParser.getVersion()
})

newGauge("CommitID", new Gauge[String] {
def value = {
AppInfoParser.getCommitId();
}
def value = AppInfoParser.getCommitId();
})

lock.synchronized {
Expand Down
Loading

0 comments on commit 3c82a0c

Please sign in to comment.