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:
Jun Rao 2012-09-14 01:29:07 +00:00
parent 2bc65dab67
commit e42c8d5f4e
6 changed files with 1 additions and 48 deletions

View File

@ -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()

View File

@ -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 + "], "

View File

@ -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
*/ */

View File

@ -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)

View File

@ -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 _ =>
}
}
} }

View File

@ -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()