-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/34103/#review85856
-----------------------------------------------------------



core/src/main/scala/kafka/controller/KafkaController.scala
<https://reviews.apache.org/r/34103/#comment137695>

    Online to Offline



core/src/main/scala/kafka/controller/KafkaController.scala
<https://reviews.apache.org/r/34103/#comment137696>

    Don't totally understand this, after line end please add some example to 
what you are proposing with example partitions, leaders, followers more of what 
is going on.



core/src/main/scala/kafka/controller/KafkaController.scala
<https://reviews.apache.org/r/34103/#comment137697>

    At this point if we can't restart partitions we are deciding here by 
throwing that the server will then get shutdown? If that is our intent we 
should do that explicitly imho.



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137698>

    I think this try / catch (and all the ditto ones after this) can become 
passed in as anon func to another reused func
    
    //something like
    def kafkaStorageCheck(f :() => Unit): Unit => {
     try {
       f()
     catch {
       case ... 
     }
    }
    
    ... then for what we have now just wrap it 
    kafkaStorageCheck(() = { 
      existingFunc()
      var exist = 2
      moreFunc()
    })
    
    will help to centralize and reason more about what exceptions are doing by 
maybe being able to eventually remove them from the application control flow. 
But, at least for now we can structure it more concisely.



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137699>

    ditto



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137700>

    ditto



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137701>

    ditto



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137702>

    ditto



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137703>

    ditto



core/src/main/scala/kafka/log/Log.scala
<https://reviews.apache.org/r/34103/#comment137704>

    ditto



core/src/main/scala/kafka/log/LogManager.scala
<https://reviews.apache.org/r/34103/#comment137705>

    ditto



core/src/main/scala/kafka/log/LogManager.scala
<https://reviews.apache.org/r/34103/#comment137706>

    ditto



core/src/main/scala/kafka/log/LogManager.scala
<https://reviews.apache.org/r/34103/#comment137707>

    I feel like this should be in its own structure? I can imaging admin 
commands to interact with this in the future and more interactions with other 
parts of the code.



core/src/main/scala/kafka/log/LogSegment.scala
<https://reviews.apache.org/r/34103/#comment137708>

    is there a way to change this to not use control flow logic and still make 
sure we aren't killing the broker?


- Joe Stein


On May 12, 2015, 12:39 p.m., Andrii Biletskyi wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34103/
> -----------------------------------------------------------
> 
> (Updated May 12, 2015, 12:39 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-2188
>     https://issues.apache.org/jira/browse/KAFKA-2188
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-2188 - JBOD Support
> 
> 
> Diffs
> -----
> 
>   core/src/main/scala/kafka/cluster/Partition.scala 
> 122b1dbbe45cb27aed79b5be1e735fb617c716b0 
>   core/src/main/scala/kafka/common/GenericKafkaStorageException.scala 
> PRE-CREATION 
>   core/src/main/scala/kafka/controller/KafkaController.scala 
> a6351163f5b6f080d6fa50bcc3533d445fcbc067 
>   core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala 
> 3b15ab4eef22c6f50a7483e99a6af40fb55aca9f 
>   core/src/main/scala/kafka/log/Log.scala 
> 84e7b8fe9dd014884b60c4fbe13c835cf02a40e4 
>   core/src/main/scala/kafka/log/LogManager.scala 
> e781ebac2677ebb22e0c1fef0cf7e5ad57c74ea4 
>   core/src/main/scala/kafka/log/LogSegment.scala 
> ed039539ac18ea4d65144073915cf112f7374631 
>   core/src/main/scala/kafka/server/KafkaApis.scala 
> 417960dd1ab407ebebad8fdb0e97415db3e91a2f 
>   core/src/main/scala/kafka/server/KafkaConfig.scala 
> 9efa15ca5567b295ab412ee9eea7c03eb4cdc18b 
>   core/src/main/scala/kafka/server/OffsetCheckpoint.scala 
> 8c5b0546908d3b3affb9f48e2ece9ed252518783 
>   core/src/main/scala/kafka/server/ReplicaFetcherThread.scala 
> b31b432a226ba79546dd22ef1d2acbb439c2e9a3 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 
> 59c9bc3ac3a8afc07a6f8c88c5871304db588d17 
>   core/src/main/scala/kafka/utils/ZkUtils.scala 
> 1da8f90b3a7abda5868186bddf221e31adbe02ce 
>   core/src/test/scala/unit/kafka/log/LogManagerTest.scala 
> 01dfbc4f8d21f6905327cd4ed6c61d657adc0143 
>   core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala 
> 60cd8249e6ec03349e20bb0a7226ea9cd66e6b17 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 
> faae0e907596a16c47e8d49a82b6a3c82797c96d 
> 
> Diff: https://reviews.apache.org/r/34103/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Andrii Biletskyi
> 
>

Reply via email to