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.api.OffsetRequest
|
||||
import kafka.log.Log._
|
||||
import kafka.common.{KafkaException, UnknownTopicOrPartitionException}
|
||||
import kafka.common.KafkaException
|
||||
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
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 {
|
||||
val d = new File(logDir, topic + "-" + partition)
|
||||
d.mkdirs()
|
||||
|
|
|
@ -34,7 +34,6 @@ import kafka.utils._
|
|||
class SocketServer(val brokerId: Int,
|
||||
val port: Int,
|
||||
val numProcessorThreads: Int,
|
||||
val monitoringPeriodSecs: Int,
|
||||
val maxQueuedRequests: Int,
|
||||
val maxRequestSize: Int = Int.MaxValue) extends Logging {
|
||||
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 */
|
||||
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 */
|
||||
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 */
|
||||
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 */
|
||||
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)
|
||||
|
||||
/* 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
|
||||
*/
|
||||
|
|
|
@ -75,7 +75,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
|
|||
socketServer = new SocketServer(config.brokerId,
|
||||
config.port,
|
||||
config.numNetworkThreads,
|
||||
config.monitoringPeriodSecs,
|
||||
config.numQueuedRequests,
|
||||
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 ",
|
||||
(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,
|
||||
port = TestUtils.choosePort,
|
||||
numProcessorThreads = 1,
|
||||
monitoringPeriodSecs = 30,
|
||||
maxQueuedRequests = 50,
|
||||
maxRequestSize = 50)
|
||||
server.startup()
|
||||
|
|
Loading…
Reference in New Issue