You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ju...@apache.org on 2012/02/28 02:50:38 UTC

svn commit: r1294441 - in /incubator/kafka/trunk/core/src: main/scala/kafka/server/KafkaConfig.scala main/scala/kafka/utils/Utils.scala test/scala/unit/kafka/log/LogManagerTest.scala

Author: junrao
Date: Tue Feb 28 01:50:38 2012
New Revision: 1294441

URL: http://svn.apache.org/viewvc?rev=1294441&view=rev
Log:
Increase maximum value of log.retention.size; patched by Elben Shira; reviewed by Jun Rao; KAFKA-285

Modified:
    incubator/kafka/trunk/core/src/main/scala/kafka/server/KafkaConfig.scala
    incubator/kafka/trunk/core/src/main/scala/kafka/utils/Utils.scala
    incubator/kafka/trunk/core/src/test/scala/unit/kafka/log/LogManagerTest.scala

Modified: incubator/kafka/trunk/core/src/main/scala/kafka/server/KafkaConfig.scala
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/core/src/main/scala/kafka/server/KafkaConfig.scala?rev=1294441&r1=1294440&r2=1294441&view=diff
==============================================================================
--- incubator/kafka/trunk/core/src/main/scala/kafka/server/KafkaConfig.scala (original)
+++ incubator/kafka/trunk/core/src/main/scala/kafka/server/KafkaConfig.scala Tue Feb 28 01:50:38 2012
@@ -65,7 +65,7 @@ class KafkaConfig(props: Properties) ext
   val logRetentionHours = Utils.getIntInRange(props, "log.retention.hours", 24 * 7, (1, Int.MaxValue))
   
   /* the maximum size of the log before deleting it */
-  val logRetentionSize = Utils.getInt(props, "log.retention.size", -1)
+  val logRetentionSize = Utils.getLong(props, "log.retention.size", -1)
 
   /* the number of hours to keep a log file before deleting it for some specific topic*/
   val logRetentionHoursMap = Utils.getTopicRentionHours(Utils.getString(props, "topic.log.retention.hours", ""))

Modified: incubator/kafka/trunk/core/src/main/scala/kafka/utils/Utils.scala
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/core/src/main/scala/kafka/utils/Utils.scala?rev=1294441&r1=1294440&r2=1294441&view=diff
==============================================================================
--- incubator/kafka/trunk/core/src/main/scala/kafka/utils/Utils.scala (original)
+++ incubator/kafka/trunk/core/src/main/scala/kafka/utils/Utils.scala Tue Feb 28 01:50:38 2012
@@ -194,6 +194,48 @@ object Utils extends Logging {
   }
   
   /**
+   * Read a required long property value or throw an exception if no such property is found
+   */
+  def getLong(props: Properties, name: String): Long = {
+    if(props.containsKey(name))
+      return getLong(props, name, -1)
+    else
+      throw new IllegalArgumentException("Missing required property '" + name + "'")
+  }
+
+  /**
+   * Read an long from the properties instance
+   * @param props The properties to read from
+   * @param name The property name
+   * @param default The default value to use if the property is not found
+   * @return the long value
+   */
+  def getLong(props: Properties, name: String, default: Long): Long = 
+    getLongInRange(props, name, default, (Long.MinValue, Long.MaxValue))
+
+  /**
+   * Read an long from the properties instance. Throw an exception 
+   * if the value is not in the given range (inclusive)
+   * @param props The properties to read from
+   * @param name The property name
+   * @param default The default value to use if the property is not found
+   * @param range The range in which the value must fall (inclusive)
+   * @throws IllegalArgumentException If the value is not in the given range
+   * @return the long value
+   */
+  def getLongInRange(props: Properties, name: String, default: Long, range: (Long, Long)): Long = {
+    val v = 
+      if(props.containsKey(name))
+        props.getProperty(name).toLong
+      else
+        default
+    if(v < range._1 || v > range._2)
+      throw new IllegalArgumentException(name + " has value " + v + " which is not in the range " + range + ".")
+    else
+      v
+  }
+
+  /**
    * Read a boolean value from the properties instance
    * @param props The properties to read from
    * @param name The property name

Modified: incubator/kafka/trunk/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
URL: http://svn.apache.org/viewvc/incubator/kafka/trunk/core/src/test/scala/unit/kafka/log/LogManagerTest.scala?rev=1294441&r1=1294440&r2=1294441&view=diff
==============================================================================
--- incubator/kafka/trunk/core/src/test/scala/unit/kafka/log/LogManagerTest.scala (original)
+++ incubator/kafka/trunk/core/src/test/scala/unit/kafka/log/LogManagerTest.scala Tue Feb 28 01:50:38 2012
@@ -107,7 +107,7 @@ class LogManagerTest extends JUnitSuite 
     config = new KafkaConfig(props) {
       override val logFileSize = (10 * (setSize - 1)).asInstanceOf[Int] // each segment will be 10 messages
       override val enableZookeeper = false
-      override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Int] // keep exactly 6 segments + 1 roll over
+      override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Long] // keep exactly 6 segments + 1 roll over
       override val logRetentionHours = retentionHours
     }
     logManager = new LogManager(config, null, time, -1, retentionMs, false)