mirror of https://github.com/apache/kafka.git
revisit broker config in 0.8; patched by Swapnil Ghike; reviewed by Jun Rao; KAFKA-325
git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8@1384624 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2bc65dab67
commit
e42c8d5f4e
|
@ -23,7 +23,7 @@ import scala.collection._
|
||||||
import kafka.server.KafkaConfig
|
import kafka.server.KafkaConfig
|
||||||
import kafka.api.OffsetRequest
|
import kafka.api.OffsetRequest
|
||||||
import kafka.log.Log._
|
import kafka.log.Log._
|
||||||
import kafka.common.{KafkaException, UnknownTopicOrPartitionException}
|
import kafka.common.KafkaException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The guy who creates and hands out logs
|
* The guy who creates and hands out logs
|
||||||
|
@ -95,12 +95,6 @@ private[kafka] class LogManager(val config: KafkaConfig,
|
||||||
* Create a log for the given topic and the given partition
|
* Create a log for the given topic and the given partition
|
||||||
*/
|
*/
|
||||||
private def createLog(topic: String, partition: Int): Log = {
|
private def createLog(topic: String, partition: Int): Log = {
|
||||||
if (partition < 0 || partition >= config.topicPartitionsMap.getOrElse(topic, numPartitions)) {
|
|
||||||
val error = "Wrong partition %d, valid partitions (0, %d)."
|
|
||||||
.format(partition, (config.topicPartitionsMap.getOrElse(topic, numPartitions) - 1))
|
|
||||||
warn(error)
|
|
||||||
throw new UnknownTopicOrPartitionException(error)
|
|
||||||
}
|
|
||||||
logCreationLock synchronized {
|
logCreationLock synchronized {
|
||||||
val d = new File(logDir, topic + "-" + partition)
|
val d = new File(logDir, topic + "-" + partition)
|
||||||
d.mkdirs()
|
d.mkdirs()
|
||||||
|
|
|
@ -34,7 +34,6 @@ import kafka.utils._
|
||||||
class SocketServer(val brokerId: Int,
|
class SocketServer(val brokerId: Int,
|
||||||
val port: Int,
|
val port: Int,
|
||||||
val numProcessorThreads: Int,
|
val numProcessorThreads: Int,
|
||||||
val monitoringPeriodSecs: Int,
|
|
||||||
val maxQueuedRequests: Int,
|
val maxQueuedRequests: Int,
|
||||||
val maxRequestSize: Int = Int.MaxValue) extends Logging {
|
val maxRequestSize: Int = Int.MaxValue) extends Logging {
|
||||||
this.logIdent = "[Socket Server on Broker " + brokerId + "], "
|
this.logIdent = "[Socket Server on Broker " + brokerId + "], "
|
||||||
|
|
|
@ -64,9 +64,6 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
/* the number of queued requests allowed before blocking the network threads */
|
/* the number of queued requests allowed before blocking the network threads */
|
||||||
val numQueuedRequests = props.getIntInRange("max.queued.requests", 500, (1, Int.MaxValue))
|
val numQueuedRequests = props.getIntInRange("max.queued.requests", 500, (1, Int.MaxValue))
|
||||||
|
|
||||||
/* the interval in which to measure performance statistics */
|
|
||||||
val monitoringPeriodSecs = props.getIntInRange("monitoring.period.secs", 600, (1, Int.MaxValue))
|
|
||||||
|
|
||||||
/* the default number of log partitions per topic */
|
/* the default number of log partitions per topic */
|
||||||
val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue))
|
val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue))
|
||||||
|
|
||||||
|
@ -112,9 +109,6 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
/* the maximum time in ms that a message in any topic is kept in memory before flushed to disk */
|
/* the maximum time in ms that a message in any topic is kept in memory before flushed to disk */
|
||||||
val defaultFlushIntervalMs = props.getInt("log.default.flush.interval.ms", flushSchedulerThreadRate)
|
val defaultFlushIntervalMs = props.getInt("log.default.flush.interval.ms", flushSchedulerThreadRate)
|
||||||
|
|
||||||
/* the number of partitions for selected topics, e.g., topic1:8,topic2:16 */
|
|
||||||
val topicPartitionsMap = Utils.getTopicPartitions(props.getString("topic.partition.count.map", ""))
|
|
||||||
|
|
||||||
/* enable auto creation of topic on the server */
|
/* enable auto creation of topic on the server */
|
||||||
val autoCreateTopics = props.getBoolean("auto.create.topics", true)
|
val autoCreateTopics = props.getBoolean("auto.create.topics", true)
|
||||||
|
|
||||||
|
@ -143,9 +137,6 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
|
||||||
|
|
||||||
val replicaMaxLagBytes = props.getLong("replica.max.lag.bytes", 4000)
|
val replicaMaxLagBytes = props.getLong("replica.max.lag.bytes", 4000)
|
||||||
|
|
||||||
/* size of the state change request queue in Zookeeper */
|
|
||||||
val stateChangeQSize = props.getInt("state.change.queue.size", 1000)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Config options relevant to a follower for a replica
|
* Config options relevant to a follower for a replica
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -75,7 +75,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
||||||
socketServer = new SocketServer(config.brokerId,
|
socketServer = new SocketServer(config.brokerId,
|
||||||
config.port,
|
config.port,
|
||||||
config.numNetworkThreads,
|
config.numNetworkThreads,
|
||||||
config.monitoringPeriodSecs,
|
|
||||||
config.numQueuedRequests,
|
config.numQueuedRequests,
|
||||||
config.maxSocketRequestSize)
|
config.maxSocketRequestSize)
|
||||||
|
|
||||||
|
|
|
@ -179,33 +179,4 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
|
||||||
assertTrue("The last flush time has to be within defaultflushInterval of current time ",
|
assertTrue("The last flush time has to be within defaultflushInterval of current time ",
|
||||||
(System.currentTimeMillis - log.getLastFlushedTime) < 100)
|
(System.currentTimeMillis - log.getLastFlushedTime) < 100)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
def testConfigurablePartitions() {
|
|
||||||
val props = TestUtils.createBrokerConfig(0, -1)
|
|
||||||
logManager.shutdown()
|
|
||||||
config = new KafkaConfig(props) {
|
|
||||||
override val logFileSize = 256
|
|
||||||
override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2")
|
|
||||||
override val flushInterval = 100
|
|
||||||
}
|
|
||||||
logManager = new LogManager(config, scheduler, time, maxRollInterval, veryLargeLogFlushInterval, maxLogAge, false)
|
|
||||||
logManager.startup
|
|
||||||
|
|
||||||
for(i <- 0 until 1) {
|
|
||||||
val log = logManager.getOrCreateLog(name, i)
|
|
||||||
for(i <- 0 until 250) {
|
|
||||||
var set = TestUtils.singleMessageSet("test".getBytes())
|
|
||||||
log.append(set)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
try
|
|
||||||
{
|
|
||||||
val log = logManager.getOrCreateLog(name, 2)
|
|
||||||
assertTrue("Should not come here", log != null)
|
|
||||||
} catch {
|
|
||||||
case _ =>
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,7 +33,6 @@ class SocketServerTest extends JUnitSuite {
|
||||||
val server: SocketServer = new SocketServer(0,
|
val server: SocketServer = new SocketServer(0,
|
||||||
port = TestUtils.choosePort,
|
port = TestUtils.choosePort,
|
||||||
numProcessorThreads = 1,
|
numProcessorThreads = 1,
|
||||||
monitoringPeriodSecs = 30,
|
|
||||||
maxQueuedRequests = 50,
|
maxQueuedRequests = 50,
|
||||||
maxRequestSize = 50)
|
maxRequestSize = 50)
|
||||||
server.startup()
|
server.startup()
|
||||||
|
|
Loading…
Reference in New Issue