mirror of https://github.com/apache/kafka.git
				
				
				
			MINOR: Use string/log interpolation instead of string concat in core and clients (#5850)
Also removed a few unused imports and tweaked the log message slightly. Reviewers: Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
		
							parent
							
								
									eede70a805
								
							
						
					
					
						commit
						9a0ea25fee
					
				|  | @ -113,8 +113,8 @@ public class CommonClientConfigs { | |||
|         HashMap<String, Object> rval = new HashMap<>(); | ||||
|         if ((!config.originals().containsKey(RECONNECT_BACKOFF_MAX_MS_CONFIG)) && | ||||
|                 config.originals().containsKey(RECONNECT_BACKOFF_MS_CONFIG)) { | ||||
|             log.debug("Disabling exponential reconnect backoff because " + RECONNECT_BACKOFF_MS_CONFIG + | ||||
|                 " is set, but " + RECONNECT_BACKOFF_MAX_MS_CONFIG + " is not."); | ||||
|             log.debug("Disabling exponential reconnect backoff because {} is set, but {} is not.", | ||||
|                     RECONNECT_BACKOFF_MS_CONFIG, RECONNECT_BACKOFF_MAX_MS_CONFIG); | ||||
|             rval.put(RECONNECT_BACKOFF_MAX_MS_CONFIG, parsedValues.get(RECONNECT_BACKOFF_MS_CONFIG)); | ||||
|         } | ||||
|         return rval; | ||||
|  |  | |||
|  | @ -334,7 +334,7 @@ public class StickyAssignor extends AbstractPartitionAssignor { | |||
|             List<TopicPartition> topicPartitions = entry.getValue(); | ||||
|             for (TopicPartition topicPartition: topicPartitions) { | ||||
|                 if (allPartitions.containsKey(topicPartition)) | ||||
|                     log.error(topicPartition + " is assigned to more than one consumer."); | ||||
|                     log.error("{} is assigned to more than one consumer.", topicPartition); | ||||
|                 allPartitions.put(topicPartition, entry.getKey()); | ||||
|             } | ||||
|         } | ||||
|  | @ -356,7 +356,8 @@ public class StickyAssignor extends AbstractPartitionAssignor { | |||
|                     String otherConsumer = allPartitions.get(topicPartition); | ||||
|                     int otherConsumerPartitionCount = currentAssignment.get(otherConsumer).size(); | ||||
|                     if (consumerPartitionCount < otherConsumerPartitionCount) { | ||||
|                         log.debug(topicPartition + " can be moved from consumer " + otherConsumer + " to consumer " + consumer + " for a more balanced assignment."); | ||||
|                         log.debug("{} can be moved from consumer {} to consumer {} for a more balanced assignment.", | ||||
|                                 topicPartition, otherConsumer, consumer); | ||||
|                         return false; | ||||
|                     } | ||||
|                 } | ||||
|  | @ -499,7 +500,7 @@ public class StickyAssignor extends AbstractPartitionAssignor { | |||
|         int currentAssignmentSize = currentPartitions.size(); | ||||
|         int maxAssignmentSize = consumer2AllPotentialPartitions.get(consumer).size(); | ||||
|         if (currentAssignmentSize > maxAssignmentSize) | ||||
|             log.error("The consumer " + consumer + " is assigned more partitions than the maximum possible."); | ||||
|             log.error("The consumer {} is assigned more partitions than the maximum possible.", consumer); | ||||
| 
 | ||||
|         if (currentAssignmentSize < maxAssignmentSize) | ||||
|             // if a consumer is not assigned all its potential partitions it is subject to reassignment | ||||
|  | @ -598,12 +599,12 @@ public class StickyAssignor extends AbstractPartitionAssignor { | |||
| 
 | ||||
|                 // the partition must have at least two consumers | ||||
|                 if (partition2AllPotentialConsumers.get(partition).size() <= 1) | ||||
|                     log.error("Expected more than one potential consumer for partition '" + partition + "'"); | ||||
|                     log.error("Expected more than one potential consumer for partition '{}'", partition); | ||||
| 
 | ||||
|                 // the partition must have a current consumer | ||||
|                 String consumer = currentPartitionConsumer.get(partition); | ||||
|                 if (consumer == null) | ||||
|                     log.error("Expected partition '" + partition + "' to be assigned to a consumer"); | ||||
|                     log.error("Expected partition '{}' to be assigned to a consumer", partition); | ||||
| 
 | ||||
|                 // check if a better-suited consumer exist for the partition; if so, reassign it | ||||
|                 for (String otherConsumer: partition2AllPotentialConsumers.get(partition)) { | ||||
|  | @ -879,7 +880,7 @@ public class StickyAssignor extends AbstractPartitionAssignor { | |||
|                 List<String> path = new ArrayList<>(Collections.singleton(pair.srcMemberId)); | ||||
|                 if (isLinked(pair.dstMemberId, pair.srcMemberId, reducedPairs, path) && !in(path, cycles)) { | ||||
|                     cycles.add(new ArrayList<>(path)); | ||||
|                     log.error("A cycle of length " + (path.size() - 1) + " was found: " + path.toString()); | ||||
|                     log.error("A cycle of length {} was found: {}", path.size() - 1, path.toString()); | ||||
|                 } | ||||
|             } | ||||
| 
 | ||||
|  | @ -896,9 +897,9 @@ public class StickyAssignor extends AbstractPartitionAssignor { | |||
|             for (Map.Entry<String, Map<ConsumerPair, Set<TopicPartition>>> topicMovements: this.partitionMovementsByTopic.entrySet()) { | ||||
|                 Set<ConsumerPair> topicMovementPairs = topicMovements.getValue().keySet(); | ||||
|                 if (hasCycles(topicMovementPairs)) { | ||||
|                     log.error("Stickiness is violated for topic " + topicMovements.getKey() | ||||
|                     log.error("Stickiness is violated for topic {}" | ||||
|                             + "\nPartition movements for this topic occurred among the following consumer pairs:" | ||||
|                             + "\n" + topicMovements.getValue().toString()); | ||||
|                             + "\n{}", topicMovements.getKey(), topicMovements.getValue().toString()); | ||||
|                     return false; | ||||
|                 } | ||||
|             } | ||||
|  |  | |||
|  | @ -196,7 +196,7 @@ public class TransactionManager { | |||
|     } | ||||
| 
 | ||||
|     TransactionManager() { | ||||
|         this(new LogContext(), null, 0, 100); | ||||
|         this(new LogContext(), null, 0, 100L); | ||||
|     } | ||||
| 
 | ||||
|     public synchronized TransactionalRequestResult initializeTransactions() { | ||||
|  |  | |||
|  | @ -102,7 +102,7 @@ public class MultiRecordsSend implements Send { | |||
|         totalWritten += totalWrittenPerCall; | ||||
| 
 | ||||
|         if (completed() && totalWritten != size) | ||||
|             log.error("mismatch in sending bytes over socket; expected: " + size + " actual: " + totalWritten); | ||||
|             log.error("mismatch in sending bytes over socket; expected: {} actual: {}", size, totalWritten); | ||||
| 
 | ||||
|         log.trace("Bytes written as part of multi-send call: {}, total bytes written so far: {}, expected bytes to write: {}", | ||||
|                 totalWrittenPerCall, totalWritten, size); | ||||
|  |  | |||
|  | @ -41,7 +41,7 @@ public class AppInfoParser { | |||
|         try (InputStream resourceStream = AppInfoParser.class.getResourceAsStream("/kafka/kafka-version.properties")) { | ||||
|             props.load(resourceStream); | ||||
|         } catch (Exception e) { | ||||
|             log.warn("Error while loading kafka-version.properties :" + e.getMessage()); | ||||
|             log.warn("Error while loading kafka-version.properties: {}", e.getMessage()); | ||||
|         } | ||||
|         VERSION = props.getProperty("version", "unknown").trim(); | ||||
|         COMMIT_ID = props.getProperty("commitId", "unknown").trim(); | ||||
|  | @ -106,8 +106,8 @@ public class AppInfoParser { | |||
|     public static class AppInfo implements AppInfoMBean { | ||||
| 
 | ||||
|         public AppInfo() { | ||||
|             log.info("Kafka version : " + AppInfoParser.getVersion()); | ||||
|             log.info("Kafka commitId : " + AppInfoParser.getCommitId()); | ||||
|             log.info("Kafka version: {}", AppInfoParser.getVersion()); | ||||
|             log.info("Kafka commitId: {}", AppInfoParser.getCommitId()); | ||||
|         } | ||||
| 
 | ||||
|         @Override | ||||
|  |  | |||
|  | @ -473,10 +473,10 @@ object AdminUtils extends Logging with AdminUtilities { | |||
|       val jsonPartitionData = zkUtils.replicaAssignmentZkData(replicaAssignment.map(e => e._1.toString -> e._2)) | ||||
| 
 | ||||
|       if (!update) { | ||||
|         info("Topic creation " + jsonPartitionData.toString) | ||||
|         info(s"Topic creation $jsonPartitionData") | ||||
|         zkUtils.createPersistentPath(zkPath, jsonPartitionData) | ||||
|       } else { | ||||
|         info("Topic update " + jsonPartitionData.toString) | ||||
|         info(s"Topic update $jsonPartitionData") | ||||
|         zkUtils.updatePersistentPath(zkPath, jsonPartitionData) | ||||
|       } | ||||
|       debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionData)) | ||||
|  |  | |||
|  | @ -29,7 +29,7 @@ import com.yammer.metrics.core.Gauge | |||
| import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0, KAFKA_2_1_IV0, KAFKA_2_1_IV1} | ||||
| import kafka.common.{MessageFormatter, OffsetAndMetadata} | ||||
| import kafka.metrics.KafkaMetricsGroup | ||||
| import kafka.server.{FetchHighWatermark, FetchLogEnd, ReplicaManager} | ||||
| import kafka.server.ReplicaManager | ||||
| import kafka.utils.CoreUtils.inLock | ||||
| import kafka.utils._ | ||||
| import kafka.zk.KafkaZkClient | ||||
|  | @ -40,7 +40,7 @@ import org.apache.kafka.common.protocol.Errors | |||
| import org.apache.kafka.common.protocol.types.Type._ | ||||
| import org.apache.kafka.common.protocol.types._ | ||||
| import org.apache.kafka.common.record._ | ||||
| import org.apache.kafka.common.requests.{IsolationLevel, OffsetCommitRequest, OffsetFetchResponse} | ||||
| import org.apache.kafka.common.requests.{OffsetCommitRequest, OffsetFetchResponse} | ||||
| import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse | ||||
| import org.apache.kafka.common.utils.{Time, Utils} | ||||
| 
 | ||||
|  |  | |||
|  | @ -33,7 +33,6 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} | |||
| import org.apache.kafka.common.internals.Topic | ||||
| import org.apache.kafka.common.protocol.Errors | ||||
| import org.apache.kafka.common.record.{FileRecords, MemoryRecords, SimpleRecord} | ||||
| import org.apache.kafka.common.requests.IsolationLevel | ||||
| import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse | ||||
| import org.apache.kafka.common.requests.TransactionResult | ||||
| import org.apache.kafka.common.utils.{Time, Utils} | ||||
|  |  | |||
|  | @ -17,7 +17,7 @@ | |||
| 
 | ||||
| package kafka.log | ||||
| 
 | ||||
| import java.io.{Closeable, File, IOException, RandomAccessFile} | ||||
| import java.io.{Closeable, File, RandomAccessFile} | ||||
| import java.nio.channels.FileChannel | ||||
| import java.nio.file.Files | ||||
| import java.nio.{ByteBuffer, MappedByteBuffer} | ||||
|  |  | |||
|  | @ -267,7 +267,7 @@ class LogCleaner(initialConfig: CleanerConfig, | |||
|    * choosing the dirtiest log, cleaning it, and then swapping in the cleaned segments. | ||||
|    */ | ||||
|   private class CleanerThread(threadId: Int) | ||||
|     extends ShutdownableThread(name = "kafka-log-cleaner-thread-" + threadId, isInterruptible = false) { | ||||
|     extends ShutdownableThread(name = s"kafka-log-cleaner-thread-$threadId", isInterruptible = false) { | ||||
| 
 | ||||
|     protected override def loggerName = classOf[LogCleaner].getName | ||||
| 
 | ||||
|  | @ -458,7 +458,7 @@ private[log] class Cleaner(val id: Int, | |||
| 
 | ||||
|   protected override def loggerName = classOf[LogCleaner].getName | ||||
| 
 | ||||
|   this.logIdent = "Cleaner " + id + ": " | ||||
|   this.logIdent = s"Cleaner $id: " | ||||
| 
 | ||||
|   /* buffer used for read i/o */ | ||||
|   private var readBuffer = ByteBuffer.allocate(ioBufferSize) | ||||
|  | @ -752,7 +752,7 @@ private[log] class Cleaner(val id: Int, | |||
|     if(readBuffer.capacity >= maxBufferSize || writeBuffer.capacity >= maxBufferSize) | ||||
|       throw new IllegalStateException("This log contains a message larger than maximum allowable size of %s.".format(maxBufferSize)) | ||||
|     val newSize = math.min(this.readBuffer.capacity * 2, maxBufferSize) | ||||
|     info("Growing cleaner I/O buffers from " + readBuffer.capacity + "bytes to " + newSize + " bytes.") | ||||
|     info(s"Growing cleaner I/O buffers from ${readBuffer.capacity} bytes to $newSize bytes.") | ||||
|     this.readBuffer = ByteBuffer.allocate(newSize) | ||||
|     this.writeBuffer = ByteBuffer.allocate(newSize) | ||||
|   } | ||||
|  |  | |||
|  | @ -59,8 +59,6 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], | |||
|                                      val logs: Pool[TopicPartition, Log], | ||||
|                                      val logDirFailureChannel: LogDirFailureChannel) extends Logging with KafkaMetricsGroup { | ||||
| 
 | ||||
|   import LogCleanerManager._ | ||||
| 
 | ||||
|   protected override def loggerName = classOf[LogCleaner].getName | ||||
| 
 | ||||
|   // package-private for testing | ||||
|  |  | |||
|  | @ -256,7 +256,7 @@ class LogManager(logDirs: Seq[File], | |||
|   private[log] def hasLogsToBeDeleted: Boolean = !logsToBeDeleted.isEmpty | ||||
| 
 | ||||
|   private def loadLog(logDir: File, recoveryPoints: Map[TopicPartition, Long], logStartOffsets: Map[TopicPartition, Long]): Unit = { | ||||
|     debug("Loading log '" + logDir.getName + "'") | ||||
|     debug(s"Loading log '${logDir.getName}'") | ||||
|     val topicPartition = Log.parseTopicPartitionName(logDir) | ||||
|     val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) | ||||
|     val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) | ||||
|  | @ -324,7 +324,7 @@ class LogManager(logDirs: Seq[File], | |||
|           recoveryPoints = this.recoveryPointCheckpoints(dir).read | ||||
|         } catch { | ||||
|           case e: Exception => | ||||
|             warn("Error occurred while reading recovery-point-offset-checkpoint file of directory " + dir, e) | ||||
|             warn(s"Error occurred while reading recovery-point-offset-checkpoint file of directory $dir", e) | ||||
|             warn("Resetting the recovery checkpoint to 0") | ||||
|         } | ||||
| 
 | ||||
|  | @ -333,7 +333,7 @@ class LogManager(logDirs: Seq[File], | |||
|           logStartOffsets = this.logStartOffsetCheckpoints(dir).read | ||||
|         } catch { | ||||
|           case e: Exception => | ||||
|             warn("Error occurred while reading log-start-offset-checkpoint file of directory " + dir, e) | ||||
|             warn(s"Error occurred while reading log-start-offset-checkpoint file of directory $dir", e) | ||||
|         } | ||||
| 
 | ||||
|         val jobsForDir = for { | ||||
|  | @ -346,7 +346,7 @@ class LogManager(logDirs: Seq[File], | |||
|             } catch { | ||||
|               case e: IOException => | ||||
|                 offlineDirs.add((dir.getAbsolutePath, e)) | ||||
|                 error("Error while loading log dir " + dir.getAbsolutePath, e) | ||||
|                 error(s"Error while loading log dir ${dir.getAbsolutePath}", e) | ||||
|             } | ||||
|           } | ||||
|         } | ||||
|  | @ -354,7 +354,7 @@ class LogManager(logDirs: Seq[File], | |||
|       } catch { | ||||
|         case e: IOException => | ||||
|           offlineDirs.add((dir.getAbsolutePath, e)) | ||||
|           error("Error while loading log dir " + dir.getAbsolutePath, e) | ||||
|           error(s"Error while loading log dir ${dir.getAbsolutePath}", e) | ||||
|       } | ||||
|     } | ||||
| 
 | ||||
|  | @ -375,7 +375,7 @@ class LogManager(logDirs: Seq[File], | |||
|       } | ||||
|     } catch { | ||||
|       case e: ExecutionException => | ||||
|         error("There was an error in one of the threads during logs loading: " + e.getCause) | ||||
|         error(s"There was an error in one of the threads during logs loading: ${e.getCause}") | ||||
|         throw e.getCause | ||||
|     } finally { | ||||
|       threadPools.foreach(_.shutdown()) | ||||
|  | @ -442,7 +442,7 @@ class LogManager(logDirs: Seq[File], | |||
| 
 | ||||
|     // close logs in each dir | ||||
|     for (dir <- liveLogDirs) { | ||||
|       debug("Flushing and closing logs at " + dir) | ||||
|       debug(s"Flushing and closing logs at $dir") | ||||
| 
 | ||||
|       val pool = Executors.newFixedThreadPool(numRecoveryThreadsPerDataDir) | ||||
|       threadPools.append(pool) | ||||
|  | @ -465,19 +465,19 @@ class LogManager(logDirs: Seq[File], | |||
|         dirJobs.foreach(_.get) | ||||
| 
 | ||||
|         // update the last flush point | ||||
|         debug("Updating recovery points at " + dir) | ||||
|         debug(s"Updating recovery points at $dir") | ||||
|         checkpointLogRecoveryOffsetsInDir(dir) | ||||
| 
 | ||||
|         debug("Updating log start offsets at " + dir) | ||||
|         debug(s"Updating log start offsets at $dir") | ||||
|         checkpointLogStartOffsetsInDir(dir) | ||||
| 
 | ||||
|         // mark that the shutdown was clean by creating marker file | ||||
|         debug("Writing clean shutdown marker at " + dir) | ||||
|         debug(s"Writing clean shutdown marker at $dir") | ||||
|         CoreUtils.swallow(Files.createFile(new File(dir, Log.CleanShutdownFile).toPath), this) | ||||
|       } | ||||
|     } catch { | ||||
|       case e: ExecutionException => | ||||
|         error("There was an error in one of the threads during LogManager shutdown: " + e.getCause) | ||||
|         error(s"There was an error in one of the threads during LogManager shutdown: ${e.getCause}") | ||||
|         throw e.getCause | ||||
|     } finally { | ||||
|       threadPools.foreach(_.shutdown()) | ||||
|  | @ -894,13 +894,13 @@ class LogManager(logDirs: Seq[File], | |||
|     try { | ||||
|       deletableLogs.foreach { | ||||
|         case (topicPartition, log) => | ||||
|           debug("Garbage collecting '" + log.name + "'") | ||||
|           debug(s"Garbage collecting '${log.name}'") | ||||
|           total += log.deleteOldSegments() | ||||
| 
 | ||||
|           val futureLog = futureLogs.get(topicPartition) | ||||
|           if (futureLog != null) { | ||||
|             // clean future logs | ||||
|             debug("Garbage collecting future log '" + futureLog.name + "'") | ||||
|             debug(s"Garbage collecting future log '${futureLog.name}'") | ||||
|             total += futureLog.deleteOldSegments() | ||||
|           } | ||||
|       } | ||||
|  | @ -910,7 +910,7 @@ class LogManager(logDirs: Seq[File], | |||
|       } | ||||
|     } | ||||
| 
 | ||||
|     debug("Log cleanup completed. " + total + " files deleted in " + | ||||
|     debug(s"Log cleanup completed. $total files deleted in " + | ||||
|                   (time.milliseconds - startMs) / 1000 + " seconds") | ||||
|   } | ||||
| 
 | ||||
|  | @ -952,13 +952,13 @@ class LogManager(logDirs: Seq[File], | |||
|     for ((topicPartition, log) <- currentLogs.toList ++ futureLogs.toList) { | ||||
|       try { | ||||
|         val timeSinceLastFlush = time.milliseconds - log.lastFlushTime | ||||
|         debug("Checking if flush is needed on " + topicPartition.topic + " flush interval  " + log.config.flushMs + | ||||
|               " last flushed " + log.lastFlushTime + " time since last flush: " + timeSinceLastFlush) | ||||
|         debug(s"Checking if flush is needed on ${topicPartition.topic} flush interval ${log.config.flushMs}" + | ||||
|               s" last flushed ${log.lastFlushTime} time since last flush: $timeSinceLastFlush") | ||||
|         if(timeSinceLastFlush >= log.config.flushMs) | ||||
|           log.flush | ||||
|       } catch { | ||||
|         case e: Throwable => | ||||
|           error("Error flushing topic " + topicPartition.topic, e) | ||||
|           error(s"Error flushing topic ${topicPartition.topic}", e) | ||||
|       } | ||||
|     } | ||||
|   } | ||||
|  |  | |||
|  | @ -20,7 +20,6 @@ package kafka.log | |||
| import java.io.File | ||||
| import java.nio.ByteBuffer | ||||
| 
 | ||||
| import kafka.common.IndexOffsetOverflowException | ||||
| import kafka.utils.CoreUtils.inLock | ||||
| import org.apache.kafka.common.errors.InvalidOffsetException | ||||
| 
 | ||||
|  |  | |||
|  | @ -133,7 +133,7 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time | |||
|         } | ||||
|       } | ||||
|     ) | ||||
|     info("Started " + acceptors.size + " acceptor threads") | ||||
|     info(s"Started ${acceptors.size} acceptor threads") | ||||
|   } | ||||
| 
 | ||||
|   /** | ||||
|  | @ -335,7 +335,7 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ | |||
|    */ | ||||
|   def close(channel: SocketChannel): Unit = { | ||||
|     if (channel != null) { | ||||
|       debug("Closing connection from " + channel.socket.getRemoteSocketAddress()) | ||||
|       debug(s"Closing connection from ${channel.socket.getRemoteSocketAddress()}") | ||||
|       connectionQuotas.dec(channel.socket.getInetAddress) | ||||
|       CoreUtils.swallow(channel.socket().close(), this, Level.ERROR) | ||||
|       CoreUtils.swallow(channel.close(), this, Level.ERROR) | ||||
|  | @ -627,7 +627,7 @@ private[kafka] class Processor(val id: Int, | |||
|         } | ||||
|       } | ||||
|     } finally { | ||||
|       debug("Closing selector - processor " + id) | ||||
|       debug(s"Closing selector - processor $id") | ||||
|       CoreUtils.swallow(closeAll(), this, Level.ERROR) | ||||
|       shutdownComplete() | ||||
|     } | ||||
|  | @ -658,7 +658,7 @@ private[kafka] class Processor(val id: Int, | |||
|             // There is no response to send to the client, we need to read more pipelined requests | ||||
|             // that are sitting in the server's socket buffer | ||||
|             updateRequestMetrics(response) | ||||
|             trace("Socket server received empty response to send, registering for read: " + response) | ||||
|             trace(s"Socket server received empty response to send, registering for read: $response") | ||||
|             // Try unmuting the channel. If there was no quota violation and the channel has not been throttled, | ||||
|             // it will be unmuted immediately. If the channel has been throttled, it will be unmuted only if the | ||||
|             // throttling delay has already passed by now. | ||||
|  |  | |||
|  | @ -161,7 +161,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|                              val tokenCache: DelegationTokenCache, | ||||
|                              val time: Time, | ||||
|                              val zkClient: KafkaZkClient) extends Logging with KafkaMetricsGroup { | ||||
|   this.logIdent = "[Token Manager on Broker " + config.brokerId + "]: " | ||||
|   this.logIdent = s"[Token Manager on Broker ${config.brokerId}]: " | ||||
| 
 | ||||
|   import DelegationTokenManager._ | ||||
| 
 | ||||
|  | @ -201,7 +201,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|   private def loadCache() { | ||||
|     lock.synchronized { | ||||
|       val tokens = zkClient.getChildren(DelegationTokensZNode.path) | ||||
|       info(s"Loading the token cache. Total token count : " + tokens.size) | ||||
|       info(s"Loading the token cache. Total token count: ${tokens.size}") | ||||
|       for (tokenId <- tokens) { | ||||
|         try { | ||||
|           getTokenFromZk(tokenId) match { | ||||
|  | @ -209,7 +209,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|             case None => | ||||
|           } | ||||
|         } catch { | ||||
|           case ex: Throwable => error(s"Error while getting Token for tokenId :$tokenId", ex) | ||||
|           case ex: Throwable => error(s"Error while getting Token for tokenId: $tokenId", ex) | ||||
|         } | ||||
|       } | ||||
|     } | ||||
|  | @ -279,7 +279,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|         val hmac = createHmac(tokenId, secretKey) | ||||
|         val token = new DelegationToken(tokenInfo, hmac) | ||||
|         updateToken(token) | ||||
|         info(s"Created a delegation token : $tokenId for owner : $owner") | ||||
|         info(s"Created a delegation token: $tokenId for owner: $owner") | ||||
|         responseCallback(CreateTokenResult(issueTimeStamp, expiryTimeStamp, maxLifeTimeStamp, tokenId, hmac, Errors.NONE)) | ||||
|       } | ||||
|     } | ||||
|  | @ -317,7 +317,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|               tokenInfo.setExpiryTimestamp(expiryTimeStamp) | ||||
| 
 | ||||
|               updateToken(token) | ||||
|               info(s"Delegation token renewed for token : " + tokenInfo.tokenId + " for owner :" + tokenInfo.owner) | ||||
|               info(s"Delegation token renewed for token: ${tokenInfo.tokenId} for owner: ${tokenInfo.owner}") | ||||
|               renewCallback(Errors.NONE, expiryTimeStamp) | ||||
|             } | ||||
|           } | ||||
|  | @ -412,7 +412,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|               expireResponseCallback(Errors.DELEGATION_TOKEN_EXPIRED, -1) | ||||
|             } else if (expireLifeTimeMs < 0) { //expire immediately | ||||
|               removeToken(tokenInfo.tokenId) | ||||
|               info(s"Token expired for token : " + tokenInfo.tokenId + " for owner :" + tokenInfo.owner) | ||||
|               info(s"Token expired for token: ${tokenInfo.tokenId} for owner: ${tokenInfo.owner}") | ||||
|               expireResponseCallback(Errors.NONE, now) | ||||
|             } else { | ||||
|               //set expiry time stamp | ||||
|  | @ -420,7 +420,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|               tokenInfo.setExpiryTimestamp(expiryTimeStamp) | ||||
| 
 | ||||
|               updateToken(token) | ||||
|               info(s"Updated expiry time for token : " + tokenInfo.tokenId + " for owner :" + tokenInfo.owner) | ||||
|               info(s"Updated expiry time for token: ${tokenInfo.tokenId} for owner: ${tokenInfo.owner}") | ||||
|               expireResponseCallback(Errors.NONE, expiryTimeStamp) | ||||
|             } | ||||
|           } | ||||
|  | @ -457,7 +457,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|       for (tokenInfo <- getAllTokenInformation) { | ||||
|         val now = time.milliseconds | ||||
|         if (tokenInfo.maxTimestamp < now || tokenInfo.expiryTimestamp < now) { | ||||
|           info(s"Delegation token expired for token : " + tokenInfo.tokenId + " for owner :" + tokenInfo.owner) | ||||
|           info(s"Delegation token expired for token: ${tokenInfo.tokenId} for owner: ${tokenInfo.owner}") | ||||
|           removeToken(tokenInfo.tokenId) | ||||
|         } | ||||
|       } | ||||
|  | @ -480,7 +480,7 @@ class DelegationTokenManager(val config: KafkaConfig, | |||
|     override def processNotification(tokenIdBytes: Array[Byte]) { | ||||
|       lock.synchronized { | ||||
|         val tokenId = new String(tokenIdBytes, StandardCharsets.UTF_8) | ||||
|         info(s"Processing Token Notification for tokenId : $tokenId") | ||||
|         info(s"Processing Token Notification for tokenId: $tokenId") | ||||
|         getTokenFromZk(tokenId) match { | ||||
|           case Some(token) => updateCache(token) | ||||
|           case None => removeCache(tokenId) | ||||
|  |  | |||
|  | @ -42,7 +42,7 @@ class ThrottledChannel(val request: RequestChannel.Request, val time: Time, val | |||
| 
 | ||||
|   // Notify the socket server that throttling has been done for this channel. | ||||
|   def notifyThrottlingDone(): Unit = { | ||||
|     trace("Channel throttled for: " + throttleTimeMs + " ms") | ||||
|     trace(s"Channel throttled for: $throttleTimeMs ms") | ||||
|     channelThrottlingCallback(new network.RequestChannel.EndThrottlingResponse(request)) | ||||
|   } | ||||
| 
 | ||||
|  |  | |||
|  | @ -359,7 +359,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { | |||
|         record.headers) | ||||
| 
 | ||||
|     override def run() { | ||||
|       info("Starting mirror maker thread " + threadName) | ||||
|       info(s"Starting mirror maker thread $threadName") | ||||
|       try { | ||||
|         consumerWrapper.init() | ||||
| 
 | ||||
|  | @ -425,7 +425,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { | |||
| 
 | ||||
|     def shutdown() { | ||||
|       try { | ||||
|         info(threadName + " shutting down") | ||||
|         info(s"$threadName shutting down") | ||||
|         shuttingDown = true | ||||
|         consumerWrapper.wakeup() | ||||
|       } | ||||
|  |  | |||
|  | @ -118,7 +118,7 @@ object ReplicaVerificationTool extends Logging { | |||
|     try Pattern.compile(regex) | ||||
|     catch { | ||||
|       case _: PatternSyntaxException => | ||||
|         throw new RuntimeException(regex + " is an invalid regex.") | ||||
|         throw new RuntimeException(s"$regex is an invalid regex.") | ||||
|     } | ||||
| 
 | ||||
|     val fetchSize = options.valueOf(fetchSizeOpt).intValue | ||||
|  | @ -199,7 +199,7 @@ object ReplicaVerificationTool extends Logging { | |||
|       } | ||||
|     }) | ||||
|     fetcherThreads.foreach(_.start()) | ||||
|     println(ReplicaVerificationTool.getCurrentTimeString() + ": verification process is started.") | ||||
|     println(s"${ReplicaVerificationTool.getCurrentTimeString()}: verification process is started.") | ||||
| 
 | ||||
|   } | ||||
| 
 | ||||
|  | @ -300,7 +300,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicPartition: Map[TopicPartitio | |||
|     debug("Begin verification") | ||||
|     maxLag = -1L | ||||
|     for ((topicPartition, fetchResponsePerReplica) <- recordsCache) { | ||||
|       debug("Verifying " + topicPartition) | ||||
|       debug(s"Verifying $topicPartition") | ||||
|       assert(fetchResponsePerReplica.size == expectedReplicasPerTopicPartition(topicPartition), | ||||
|         "fetched " + fetchResponsePerReplica.size + " replicas for " + topicPartition + ", but expected " | ||||
|           + expectedReplicasPerTopicPartition(topicPartition) + " replicas") | ||||
|  |  | |||
|  | @ -147,7 +147,7 @@ object CoreUtils extends Logging { | |||
|       } | ||||
|     } catch { | ||||
|       case e: Exception => { | ||||
|         error("Failed to register Mbean " + name, e) | ||||
|         error(s"Failed to register Mbean $name", e) | ||||
|         false | ||||
|       } | ||||
|     } | ||||
|  |  | |||
|  | @ -36,7 +36,7 @@ class FileLock(val file: File) extends Logging { | |||
|    */ | ||||
|   def lock() { | ||||
|     this synchronized { | ||||
|       trace("Acquiring lock on " + file.getAbsolutePath) | ||||
|       trace(s"Acquiring lock on ${file.getAbsolutePath}") | ||||
|       flock = channel.lock() | ||||
|     } | ||||
|   } | ||||
|  | @ -46,7 +46,7 @@ class FileLock(val file: File) extends Logging { | |||
|    */ | ||||
|   def tryLock(): Boolean = { | ||||
|     this synchronized { | ||||
|       trace("Acquiring lock on " + file.getAbsolutePath) | ||||
|       trace(s"Acquiring lock on ${file.getAbsolutePath}") | ||||
|       try { | ||||
|         // weirdly this method will return null if the lock is held by another | ||||
|         // process, but will throw an exception if the lock is held by this process | ||||
|  | @ -64,7 +64,7 @@ class FileLock(val file: File) extends Logging { | |||
|    */ | ||||
|   def unlock() { | ||||
|     this synchronized { | ||||
|       trace("Releasing lock on " + file.getAbsolutePath) | ||||
|       trace(s"Releasing lock on ${file.getAbsolutePath}") | ||||
|       if(flock != null) | ||||
|         flock.release() | ||||
|     } | ||||
|  |  | |||
|  | @ -113,7 +113,7 @@ class KafkaScheduler(val threads: Int, | |||
|           trace("Beginning execution of scheduled task '%s'.".format(name)) | ||||
|           fun() | ||||
|         } catch { | ||||
|           case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) | ||||
|           case t: Throwable => error(s"Uncaught exception in scheduled task '$name'", t) | ||||
|         } finally { | ||||
|           trace("Completed execution of scheduled task '%s'.".format(name)) | ||||
|         } | ||||
|  |  | |||
|  | @ -220,7 +220,7 @@ class ZkUtils(val zkClient: ZkClient, | |||
|           + "Probably this controller is still using the old format [%s] to store the broker id in zookeeper".format(controllerInfoString)) | ||||
|         try controllerInfoString.toInt | ||||
|         catch { | ||||
|           case t: Throwable => throw new KafkaException("Failed to parse the controller info: " + controllerInfoString + ". This is neither the new or the old format.", t) | ||||
|           case t: Throwable => throw new KafkaException(s"Failed to parse the controller info: $controllerInfoString. This is neither the new or the old format.", t) | ||||
|         } | ||||
|     } | ||||
|   } | ||||
|  | @ -415,11 +415,11 @@ class ZkUtils(val zkClient: ZkClient, | |||
|           case _: ZkNoNodeException => // the node disappeared; treat as if node existed and let caller handles this | ||||
|         } | ||||
|         if (storedData == null || storedData != data) { | ||||
|           info("conflict in " + path + " data: " + data + " stored data: " + storedData) | ||||
|           info(s"conflict in $path data: $data stored data: $storedData") | ||||
|           throw e | ||||
|         } else { | ||||
|           // otherwise, the creation succeeded, return normally | ||||
|           info(path + " exists with value " + data + " during connection loss; this is ok") | ||||
|           info(s"$path exists with value $data during connection loss; this is ok") | ||||
|         } | ||||
|     } | ||||
|   } | ||||
|  |  | |||
|  | @ -25,7 +25,7 @@ import kafka.network.SocketServer | |||
| import kafka.security.auth._ | ||||
| import kafka.server.{BaseRequestTest, KafkaConfig} | ||||
| import kafka.utils.TestUtils | ||||
| import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewPartitions} | ||||
| import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig} | ||||
| import org.apache.kafka.clients.consumer._ | ||||
| import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener | ||||
| import org.apache.kafka.clients.producer._ | ||||
|  |  | |||
|  | @ -18,7 +18,7 @@ | |||
| package kafka.api | ||||
| 
 | ||||
| import com.yammer.metrics.Metrics | ||||
| import com.yammer.metrics.core.{Gauge, Metric, MetricName} | ||||
| import com.yammer.metrics.core.Gauge | ||||
| 
 | ||||
| import java.io.File | ||||
| import java.util.ArrayList | ||||
|  |  | |||
|  | @ -23,7 +23,7 @@ import org.apache.kafka.common.TopicPartition | |||
| import org.apache.kafka.common.errors.FencedLeaderEpochException | ||||
| import org.apache.kafka.common.protocol.Errors | ||||
| import org.apache.kafka.common.record.MemoryRecords | ||||
| import org.apache.kafka.common.requests.{FetchRequest, IsolationLevel} | ||||
| import org.apache.kafka.common.requests.FetchRequest | ||||
| import org.easymock.{EasyMock, EasyMockSupport} | ||||
| import org.junit.Test | ||||
| import org.junit.Assert._ | ||||
|  |  | |||
|  | @ -25,7 +25,6 @@ import kafka.server.{BrokerTopicStats, LogDirFailureChannel} | |||
| import kafka.utils._ | ||||
| import org.apache.kafka.clients.consumer.OffsetOutOfRangeException | ||||
| import org.apache.kafka.common.record.FileRecords | ||||
| import org.apache.kafka.common.requests.IsolationLevel | ||||
| import org.apache.kafka.common.utils.Utils | ||||
| 
 | ||||
| /** | ||||
|  |  | |||
|  | @ -28,7 +28,6 @@ import org.apache.kafka.common.TopicPartition | |||
| import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME | ||||
| import org.apache.kafka.common.protocol.Errors | ||||
| import org.apache.kafka.common.record._ | ||||
| import org.apache.kafka.common.requests.IsolationLevel | ||||
| import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse | ||||
| import org.apache.kafka.common.requests.TransactionResult | ||||
| import org.apache.kafka.common.utils.MockTime | ||||
|  |  | |||
|  | @ -17,7 +17,6 @@ | |||
| 
 | ||||
| package kafka.server | ||||
| 
 | ||||
| import java.lang.{Long => JLong} | ||||
| import java.net.InetAddress | ||||
| import java.util | ||||
| import java.util.{Collections, Optional} | ||||
|  |  | |||
|  | @ -22,7 +22,6 @@ import java.util.Properties | |||
| import kafka.api.{ApiVersion, KAFKA_0_8_2} | ||||
| import kafka.cluster.EndPoint | ||||
| import kafka.message._ | ||||
| import kafka.metrics.KafkaMetricsConfig | ||||
| import kafka.utils.{CoreUtils, TestUtils} | ||||
| import org.apache.kafka.common.config.ConfigException | ||||
| import org.apache.kafka.common.metrics.Sensor | ||||
|  |  | |||
|  | @ -30,7 +30,6 @@ import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRec | |||
| import org.apache.kafka.common.requests.FetchRequest.PartitionData | ||||
| import org.easymock.EasyMock | ||||
| import EasyMock._ | ||||
| import org.apache.kafka.common.requests.IsolationLevel | ||||
| import org.junit.Assert._ | ||||
| import org.junit.{After, Test} | ||||
| 
 | ||||
|  |  | |||
|  | @ -32,7 +32,6 @@ import java.util.concurrent.atomic.AtomicBoolean | |||
| 
 | ||||
| import org.apache.kafka.common.TopicPartition | ||||
| import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} | ||||
| import org.apache.kafka.common.requests.IsolationLevel | ||||
| import org.easymock.EasyMock | ||||
| import org.junit.Assert._ | ||||
| 
 | ||||
|  |  | |||
|  | @ -19,7 +19,6 @@ package kafka.server.epoch | |||
| 
 | ||||
| import java.io.File | ||||
| 
 | ||||
| import kafka.server.LogOffsetMetadata | ||||
| import kafka.server.checkpoints.{LeaderEpochCheckpoint, LeaderEpochCheckpointFile} | ||||
| import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} | ||||
| import kafka.utils.TestUtils | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue