mirror of https://github.com/apache/kafka.git
kafka-649; Cleanup log4j logging; patched by Jun Rao; reviewed by Jay Kreps
This commit is contained in:
parent
ef123c20b8
commit
ae362b0864
|
|
@ -220,7 +220,8 @@ class Partition(val topic: String,
|
|||
if (!inSyncReplicas.contains(replica) && replica.logEndOffset >= leaderHW) {
|
||||
// expand ISR
|
||||
val newInSyncReplicas = inSyncReplicas + replica
|
||||
info("Expanding ISR for topic %s partition %d to %s".format(topic, partitionId, newInSyncReplicas.map(_.brokerId).mkString(", ")))
|
||||
info("Expanding ISR for topic %s partition %d from %s to %s"
|
||||
.format(topic, partitionId, inSyncReplicas.map(_.brokerId).mkString(","), newInSyncReplicas.map(_.brokerId).mkString(",")))
|
||||
// update ISR in ZK and cache
|
||||
updateIsr(newInSyncReplicas)
|
||||
replicaManager.isrExpandRate.mark()
|
||||
|
|
@ -315,7 +316,7 @@ class Partition(val topic: String,
|
|||
}
|
||||
|
||||
private def updateIsr(newIsr: Set[Replica]) {
|
||||
info("Updated ISR for topic %s partition %d to %s".format(topic, partitionId, newIsr.mkString(", ")))
|
||||
debug("Updated ISR for topic %s partition %d to %s".format(topic, partitionId, newIsr.mkString(",")))
|
||||
val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion)
|
||||
// use the epoch of the controller that made the leadership decision, instead of the current controller epoch
|
||||
val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient,
|
||||
|
|
|
|||
|
|
@ -109,7 +109,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
|
|||
|
||||
def clearCurrentChunk() {
|
||||
try {
|
||||
info("Clearing the current data chunk for this consumer iterator")
|
||||
debug("Clearing the current data chunk for this consumer iterator")
|
||||
current.set(null)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -53,8 +53,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
|
|||
val currentLeaderIsrZkPathVersion = currentLeaderAndIsr.zkVersion
|
||||
val newLeaderAndIsr = liveBrokersInIsr.isEmpty match {
|
||||
case true =>
|
||||
debug("No broker is ISR is alive, picking the leader from the alive assigned replicas: %s"
|
||||
.format(liveAssignedReplicasToThisPartition.mkString(",")))
|
||||
debug("No broker in ISR is alive for %s. Pick the leader from the alive assigned replicas: %s"
|
||||
.format(topicAndPartition, liveAssignedReplicasToThisPartition.mkString(",")))
|
||||
liveAssignedReplicasToThisPartition.isEmpty match {
|
||||
case true =>
|
||||
throw new NoReplicaOnlineException(("No replica for partition " +
|
||||
|
|
@ -63,13 +63,13 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext) exten
|
|||
case false =>
|
||||
ControllerStats.uncleanLeaderElectionRate.mark()
|
||||
val newLeader = liveAssignedReplicasToThisPartition.head
|
||||
warn("No broker in ISR is alive, elected leader from the alive replicas is [%s], ".format(newLeader) +
|
||||
"There's potential data loss")
|
||||
warn("No broker in ISR is alive for %s. Elect leader from broker %s. There's potential data loss."
|
||||
.format(topicAndPartition, newLeader))
|
||||
new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1)
|
||||
}
|
||||
case false =>
|
||||
val newLeader = liveBrokersInIsr.head
|
||||
debug("Some broker in ISR is alive, selecting the leader from the ISR: " + newLeader)
|
||||
debug("Some broker in ISR is alive for %s. Select %d from ISR to be the leader.".format(topicAndPartition, newLeader))
|
||||
new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1)
|
||||
}
|
||||
info("Selected new leader and ISR %s for offline partition %s".format(newLeaderAndIsr.toString(), topicAndPartition))
|
||||
|
|
|
|||
|
|
@ -44,7 +44,7 @@ class FileMessageSet private[kafka](val file: File,
|
|||
private val _size = new AtomicInteger(scala.math.min(channel.size().toInt, limit) - start)
|
||||
|
||||
if (initChannelPositionToEnd) {
|
||||
info("Creating or reloading log segment %s".format(file.getAbsolutePath))
|
||||
debug("Creating or reloading log segment %s".format(file.getAbsolutePath))
|
||||
/* set the file position to the last byte in the file */
|
||||
channel.position(channel.size)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -127,7 +127,7 @@ private[kafka] class Log(val dir: File,
|
|||
/* Calculate the offset of the next message */
|
||||
private var nextOffset: AtomicLong = new AtomicLong(segments.view.last.nextOffset())
|
||||
|
||||
debug("Completed load of log %s with log end offset %d".format(name, logEndOffset))
|
||||
info("Completed load of log %s with log end offset %d".format(name, logEndOffset))
|
||||
|
||||
newGauge(name + "-" + "NumLogSegments",
|
||||
new Gauge[Int] { def getValue = numberOfSegments })
|
||||
|
|
|
|||
|
|
@ -90,7 +90,7 @@ class OffsetIndex(val file: File, val baseOffset: Long, val maxIndexSize: Int =
|
|||
/* the last offset in the index */
|
||||
var lastOffset = readLastOffset()
|
||||
|
||||
info("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d"
|
||||
debug("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d"
|
||||
.format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset, mmap.position))
|
||||
|
||||
/* the maximum number of entries this index can hold */
|
||||
|
|
|
|||
|
|
@ -135,7 +135,7 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
|
|||
} else {
|
||||
// currently, if in async mode, we just log the serialization error. We need to revisit
|
||||
// this when doing kafka-496
|
||||
error("Error serializing message ", t)
|
||||
error("Error serializing message for topic %s".format(e.topic), t)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -205,10 +205,10 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
Runtime.getRuntime.halt(1)
|
||||
null
|
||||
case utpe: UnknownTopicOrPartitionException =>
|
||||
warn(utpe.getMessage)
|
||||
warn("Produce request: " + utpe.getMessage)
|
||||
new ProduceResult(topicAndPartition, utpe)
|
||||
case nle: NotLeaderForPartitionException =>
|
||||
warn(nle.getMessage)
|
||||
warn("Produce request: " + nle.getMessage)
|
||||
new ProduceResult(topicAndPartition, nle)
|
||||
case e =>
|
||||
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark()
|
||||
|
|
@ -291,15 +291,17 @@ class KafkaApis(val requestChannel: RequestChannel,
|
|||
// since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request
|
||||
// for a partition it is the leader for
|
||||
case utpe: UnknownTopicOrPartitionException =>
|
||||
warn(utpe.getMessage)
|
||||
warn("Fetch request: " + utpe.getMessage)
|
||||
new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
|
||||
case nle: NotLeaderForPartitionException =>
|
||||
warn(nle.getMessage)
|
||||
warn("Fetch request: " + nle.getMessage)
|
||||
new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
|
||||
case t =>
|
||||
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
|
||||
BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark()
|
||||
error("error when processing request " + (topic, partition, offset, fetchSize), t)
|
||||
error("Error when processing fetch request for topic %s partition %d offset %d from %s with correlation id %d"
|
||||
.format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId),
|
||||
t)
|
||||
new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty)
|
||||
}
|
||||
(TopicAndPartition(topic, partition), partitionData)
|
||||
|
|
|
|||
|
|
@ -131,7 +131,7 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
def stopReplicas(stopReplicaRequest: StopReplicaRequest): (mutable.Map[(String, Int), Short], Short) = {
|
||||
val responseMap = new collection.mutable.HashMap[(String, Int), Short]
|
||||
if(stopReplicaRequest.controllerEpoch < controllerEpoch) {
|
||||
stateChangeLogger.error("Broker %d received stop replica request from an old controller epoch %d."
|
||||
stateChangeLogger.warn("Broker %d received stop replica request from an old controller epoch %d."
|
||||
.format(localBrokerId, stopReplicaRequest.controllerEpoch) +
|
||||
" Latest known controller epoch is %d " + controllerEpoch)
|
||||
(responseMap, ErrorMapping.StaleControllerEpochCode)
|
||||
|
|
@ -196,14 +196,14 @@ class ReplicaManager(val config: KafkaConfig,
|
|||
|
||||
def becomeLeaderOrFollower(leaderAndISRRequest: LeaderAndIsrRequest): (collection.Map[(String, Int), Short], Short) = {
|
||||
leaderAndISRRequest.partitionStateInfos.foreach(p =>
|
||||
stateChangeLogger.trace("Broker %d handling LeaderAndIsr request correlationId %d received from controller %d epoch %d for partition [%s,%d]"
|
||||
stateChangeLogger.trace("Broker %d handling LeaderAndIsr request correlation id %d received from controller %d epoch %d for partition [%s,%d]"
|
||||
.format(localBrokerId, leaderAndISRRequest.correlationId, leaderAndISRRequest.controllerId,
|
||||
leaderAndISRRequest.controllerEpoch, p._1._1, p._1._2)))
|
||||
info("Handling LeaderAndIsr request %s".format(leaderAndISRRequest))
|
||||
|
||||
val responseMap = new collection.mutable.HashMap[(String, Int), Short]
|
||||
if(leaderAndISRRequest.controllerEpoch < controllerEpoch) {
|
||||
stateChangeLogger.error("Broker %d received LeaderAndIsr request correlationId %d with an old controllerEpoch %d, latest known controllerEpoch is %d"
|
||||
stateChangeLogger.warn("Broker %d received LeaderAndIsr request correlation id %d with an old controller epoch %d. Latest known controller epoch is %d"
|
||||
.format(localBrokerId, leaderAndISRRequest.controllerEpoch, leaderAndISRRequest.correlationId, controllerEpoch))
|
||||
(responseMap, ErrorMapping.StaleControllerEpochCode)
|
||||
}else {
|
||||
|
|
|
|||
|
|
@ -328,12 +328,12 @@ object ZkUtils extends Logging {
|
|||
def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = {
|
||||
try {
|
||||
val stat = client.writeData(path, data, expectVersion)
|
||||
info("Conditional update of zkPath %s with value %s and expected version %d succeeded, returning the new version: %d"
|
||||
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
|
||||
.format(path, data, expectVersion, stat.getVersion))
|
||||
(true, stat.getVersion)
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
error("Conditional update of zkPath %s with data %s and expected version %d failed".format(path, data,
|
||||
error("Conditional update of path %s with data %s and expected version %d failed".format(path, data,
|
||||
expectVersion), e)
|
||||
(false, -1)
|
||||
}
|
||||
|
|
@ -346,13 +346,13 @@ object ZkUtils extends Logging {
|
|||
def conditionalUpdatePersistentPathIfExists(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = {
|
||||
try {
|
||||
val stat = client.writeData(path, data, expectVersion)
|
||||
info("Conditional update of zkPath %s with value %s and expected version %d succeeded, returning the new version: %d"
|
||||
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
|
||||
.format(path, data, expectVersion, stat.getVersion))
|
||||
(true, stat.getVersion)
|
||||
} catch {
|
||||
case nne: ZkNoNodeException => throw nne
|
||||
case e: Exception =>
|
||||
error("Conditional update of zkPath %s with data %s and expected version %d failed".format(path, data,
|
||||
error("Conditional update of path %s with data %s and expected version %d failed".format(path, data,
|
||||
expectVersion), e)
|
||||
(false, -1)
|
||||
}
|
||||
|
|
|
|||
Loading…
Reference in New Issue