[jira] [Commented] (KAFKA-974) can't use public release maven repo because of failure of downloaded dependency

2013-11-28 Thread Michael Noll (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-974?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13834655#comment-13834655
 ] 

Michael Noll commented on KAFKA-974:


FWIW here is an sbt snippet that resolves this problem.  Chris' last comment 
was pointing me in the right direction but I still had to figure out the 
correct sbt syntax -- literally using exclude("jms", "jmxtools", "jmxri") did 
not work.

{code}
libraryDependencies ++= Seq(
  "org.apache.kafka" % "kafka_2.10" % "0.8.0"
exclude("javax.jms", "jms")
exclude("com.sun.jdmk", "jmxtools")
exclude("com.sun.jmx", "jmxri"),
  // Alternatively, this apparently also works but it will exclude ALL deps of 
the excluded organizations:
  //"org.apache.kafka" % "kafka_2.10" % "0.8.0" excludeAll(
  //  ExclusionRule(organization = "com.sun.jdmk"),
  //  ExclusionRule(organization = "com.sun.jmx"),
  //  ExclusionRule(organization = "javax.jms")
  //),
  ...other dependencies...
)
{code}

Versions:
- sbt 0.13.0
- Scala 2.10.3

> can't use public release maven repo because of failure of downloaded 
> dependency
> ---
>
> Key: KAFKA-974
> URL: https://issues.apache.org/jira/browse/KAFKA-974
> Project: Kafka
>  Issue Type: Bug
>Reporter: Joe Stein
> Fix For: 0.8
>
> Attachments: KAFKA-974.patch, KAFKA-974.v2.patch
>
>
> trying to use the 0.8.0-beta1 release from public maven
> name := "Stub"
> version := "1.0.0.0"
> scalaVersion := "2.9.2"
> mainClass := Some("Stub")
> libraryDependencies ++= Seq(
>   "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1"
> )
> results in
> Joes-MacBook-Air:stub joestein$ ./sbt compile
> [info] Set current project to default-63d5f2 (in build 
> file:/opt/medialets/SymanticManager/scala/stub/)
> [info] Updating 
> {file:/opt/medialets/SymanticManager/scala/stub/}default-63d5f2...
> [warn][NOT FOUND  ] javax.jms#jms;1.1!jms.jar (50ms)
> [warn]  public: tried
> [warn]   http://repo1.maven.org/maven2/javax/jms/jms/1.1/jms-1.1.jar
> [warn][NOT FOUND  ] com.sun.jdmk#jmxtools;1.2.1!jmxtools.jar (12ms)
> [warn]  public: tried
> [warn]   
> http://repo1.maven.org/maven2/com/sun/jdmk/jmxtools/1.2.1/jmxtools-1.2.1.jar
> [warn][NOT FOUND  ] com.sun.jmx#jmxri;1.2.1!jmxri.jar (71ms)
> [warn]  public: tried
> [warn]   http://repo1.maven.org/maven2/com/sun/jmx/jmxri/1.2.1/jmxri-1.2.1.jar
> [warn]::
> [warn]::  FAILED DOWNLOADS::
> [warn]:: ^ see resolution messages for details  ^ ::
> [warn]::
> [warn]:: javax.jms#jms;1.1!jms.jar
> [warn]:: com.sun.jdmk#jmxtools;1.2.1!jmxtools.jar
> [warn]:: com.sun.jmx#jmxri;1.2.1!jmxri.jar
> [warn]::
> [info] 
> [warn] :: problems summary ::
> [warn]  WARNINGS
> [warn][NOT FOUND  ] javax.jms#jms;1.1!jms.jar (50ms)
> [warn] public: tried
> [warn]  http://repo1.maven.org/maven2/javax/jms/jms/1.1/jms-1.1.jar
> [warn][NOT FOUND  ] com.sun.jdmk#jmxtools;1.2.1!jmxtools.jar 
> (12ms)
> [warn] public: tried
> [warn]  
> http://repo1.maven.org/maven2/com/sun/jdmk/jmxtools/1.2.1/jmxtools-1.2.1.jar
> [warn][NOT FOUND  ] com.sun.jmx#jmxri;1.2.1!jmxri.jar (71ms)
> [warn] public: tried
> [warn]  
> http://repo1.maven.org/maven2/com/sun/jmx/jmxri/1.2.1/jmxri-1.2.1.jar
> [warn]::
> [warn]::  FAILED DOWNLOADS::
> [warn]:: ^ see resolution messages for details  ^ ::
> [warn]::
> [warn]:: javax.jms#jms;1.1!jms.jar
> [warn]:: com.sun.jdmk#jmxtools;1.2.1!jmxtools.jar
> [warn]:: com.sun.jmx#jmxri;1.2.1!jmxri.jar
> [warn]::
> [info] 
> [info] :: USE VERBOSE OR DEBUG MESSAGE LEVEL FOR MORE DETAILS
> [error] 
> {file:/opt/medialets/SymanticManager/scala/stub/}default-63d5f2/*:update: 
> sbt.ResolveException: download failed: javax.jms#jms;1.1!jms.jar
> [error] download failed: com.sun.jdmk#jmxtools;1.2.1!jmxtools.jar
> [error] download failed: com.sun.jmx#jmxri;1.2.1!jmxri.jar
> [error] Total time: 3 s, completed Jul 13, 2013 1:55:36 AM



--
This message was sent by Atlassian JIRA
(v6.1#6144)


Re: Review Request 15901: ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Jun Rao

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



core/src/main/scala/kafka/server/ReplicaManager.scala


We probably should guard the more general case, i.e, if can't find the 
leader broker, we will just skip this partition (rather than just handling the 
leader = -1 case).

It's probably also useful to log the partitions that are skipped.



- Jun Rao


On Nov. 28, 2013, 6:36 a.m., Swapnil Ghike wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/15901/
> ---
> 
> (Updated Nov. 28, 2013, 6:36 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1152
> https://issues.apache.org/jira/browse/KAFKA-1152
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 
> 161f58134f20f9335dbd2bee6ac3f71897cbef7c 
> 
> Diff: https://reviews.apache.org/r/15901/diff/
> 
> 
> Testing
> ---
> 
> Builds with all scala versions; unit tests pass
> 
> 
> Thanks,
> 
> Swapnil Ghike
> 
>



Re: Review Request 15901: Patch for KAFKA-1152

2013-11-28 Thread Swapnil Ghike

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

(Updated Nov. 28, 2013, 6:19 p.m.)


Review request for kafka.


Summary (updated)
-

Patch for KAFKA-1152


Bugs: KAFKA-1152
https://issues.apache.org/jira/browse/KAFKA-1152


Repository: kafka


Description
---

ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
leader == -1


Diffs (updated)
-

  core/src/main/scala/kafka/server/ReplicaManager.scala 
161f58134f20f9335dbd2bee6ac3f71897cbef7c 

Diff: https://reviews.apache.org/r/15901/diff/


Testing
---

Builds with all scala versions; unit tests pass


Thanks,

Swapnil Ghike



[jira] [Updated] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Swapnil Ghike updated KAFKA-1152:
-

Attachment: KAFKA-1152_2013-11-28_10:19:05.patch

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152_2013-11-28_10:19:05.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13835022#comment-13835022
 ] 

Swapnil Ghike commented on KAFKA-1152:
--

Updated reviewboard https://reviews.apache.org/r/15901/
 against branch origin/trunk

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152_2013-11-28_10:19:05.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] Subscription: outstanding kafka patches

2013-11-28 Thread jira
Issue Subscription
Filter: outstanding kafka patches (75 issues)
The list of outstanding kafka patches
Subscriber: kafka-mailing-list

Key Summary
KAFKA-1152  ReplicaManager's handling of the leaderAndIsrRequest should 
gracefully handle leader == -1
https://issues.apache.org/jira/browse/KAFKA-1152
KAFKA-1145  Broker fail to sync after restart
https://issues.apache.org/jira/browse/KAFKA-1145
KAFKA-1144  commitOffsets can be passed the offsets to commit
https://issues.apache.org/jira/browse/KAFKA-1144
KAFKA-1142  Patch review tool should take diff with origin from last divergent 
point
https://issues.apache.org/jira/browse/KAFKA-1142
KAFKA-1130  "log.dirs" is a confusing property name
https://issues.apache.org/jira/browse/KAFKA-1130
KAFKA-1116  Need to upgrade sbt-assembly to compile on scala 2.10.2
https://issues.apache.org/jira/browse/KAFKA-1116
KAFKA-1110  Unable to produce messages with snappy/gzip compression
https://issues.apache.org/jira/browse/KAFKA-1110
KAFKA-1109  Need to fix GC log configuration code, not able to override 
KAFKA_GC_LOG_OPTS
https://issues.apache.org/jira/browse/KAFKA-1109
KAFKA-1106  HighwaterMarkCheckpoint failure puting broker into a bad state
https://issues.apache.org/jira/browse/KAFKA-1106
KAFKA-1093  Log.getOffsetsBefore(t, …) does not return the last confirmed 
offset before t
https://issues.apache.org/jira/browse/KAFKA-1093
KAFKA-1086  Improve GetOffsetShell to find metadata automatically
https://issues.apache.org/jira/browse/KAFKA-1086
KAFKA-1082  zkclient dies after UnknownHostException in zk reconnect
https://issues.apache.org/jira/browse/KAFKA-1082
KAFKA-1079  Liars in PrimitiveApiTest that promise to test api in compression 
mode, but don't do this actually
https://issues.apache.org/jira/browse/KAFKA-1079
KAFKA-1074  Reassign partitions should delete the old replicas from disk
https://issues.apache.org/jira/browse/KAFKA-1074
KAFKA-1049  Encoder implementations are required to provide an undocumented 
constructor.
https://issues.apache.org/jira/browse/KAFKA-1049
KAFKA-1032  Messages sent to the old leader will be lost on broker GC resulted 
failure
https://issues.apache.org/jira/browse/KAFKA-1032
KAFKA-1020  Remove getAllReplicasOnBroker from KafkaController
https://issues.apache.org/jira/browse/KAFKA-1020
KAFKA-1012  Implement an Offset Manager and hook offset requests to it
https://issues.apache.org/jira/browse/KAFKA-1012
KAFKA-1011  Decompression and re-compression on MirrorMaker could result in 
messages being dropped in the pipeline
https://issues.apache.org/jira/browse/KAFKA-1011
KAFKA-1005  kafka.perf.ConsumerPerformance not shutting down consumer
https://issues.apache.org/jira/browse/KAFKA-1005
KAFKA-998   Producer should not retry on non-recoverable error codes
https://issues.apache.org/jira/browse/KAFKA-998
KAFKA-997   Provide a strict verification mode when reading configuration 
properties
https://issues.apache.org/jira/browse/KAFKA-997
KAFKA-996   Capitalize first letter for log entries
https://issues.apache.org/jira/browse/KAFKA-996
KAFKA-984   Avoid a full rebalance in cases when a new topic is discovered but 
container/broker set stay the same
https://issues.apache.org/jira/browse/KAFKA-984
KAFKA-976   Order-Preserving Mirror Maker Testcase
https://issues.apache.org/jira/browse/KAFKA-976
KAFKA-967   Use key range in ProducerPerformance
https://issues.apache.org/jira/browse/KAFKA-967
KAFKA-917   Expose zk.session.timeout.ms in console consumer
https://issues.apache.org/jira/browse/KAFKA-917
KAFKA-885   sbt package builds two kafka jars
https://issues.apache.org/jira/browse/KAFKA-885
KAFKA-881   Kafka broker not respecting log.roll.hours
https://issues.apache.org/jira/browse/KAFKA-881
KAFKA-873   Consider replacing zkclient with curator (with zkclient-bridge)
https://issues.apache.org/jira/browse/KAFKA-873
KAFKA-868   System Test - add test case for rolling controlled shutdown
https://issues.apache.org/jira/browse/KAFKA-868
KAFKA-863   System Test - update 0.7 version of kafka-run-class.sh for 
Migration Tool test cases
https://issues.apache.org/jira/browse/KAFKA-863
KAFKA-859   support basic auth protection of mx4j console
https://issues.apache.org/jira/browse/KAFKA-859
KAFKA-855   Ant+Ivy build for Kafka
https://issues.apache.org/jira/browse/KAFKA-855
KAFKA-854   Upgrade dependencies for 0.8
https://issues.apache.org/jira/browse/KAFKA-854
KAFKA-815   Improve SimpleConsumerShell to take in a max messages config option
https://issues.apache.org/jira/browse/KAFKA-815
KAFKA-745   Remove getShutdownReceive() and other kafka sp

Re: Review Request 15901: Patch for KAFKA-1152

2013-11-28 Thread Jun Rao

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



core/src/main/scala/kafka/server/ReplicaManager.scala


Could we use case match here? Then, we don't need to call leaders.find() 
again.



core/src/main/scala/kafka/server/ReplicaManager.scala


Could we use map.put() since it's used in other places?



core/src/main/scala/kafka/server/ReplicaManager.scala


The ordering of leader and partition is incorrect.


- Jun Rao


On Nov. 28, 2013, 6:19 p.m., Swapnil Ghike wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/15901/
> ---
> 
> (Updated Nov. 28, 2013, 6:19 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1152
> https://issues.apache.org/jira/browse/KAFKA-1152
> 
> 
> Repository: kafka
> 
> 
> Description
> ---
> 
> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> 
> 
> Diffs
> -
> 
>   core/src/main/scala/kafka/server/ReplicaManager.scala 
> 161f58134f20f9335dbd2bee6ac3f71897cbef7c 
> 
> Diff: https://reviews.apache.org/r/15901/diff/
> 
> 
> Testing
> ---
> 
> Builds with all scala versions; unit tests pass
> 
> 
> Thanks,
> 
> Swapnil Ghike
> 
>



[jira] [Commented] (KAFKA-717) scala 2.10 build support

2013-11-28 Thread Martin Eigenbrodt (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13835183#comment-13835183
 ] 

Martin Eigenbrodt commented on KAFKA-717:
-

Isn't this duplicated and fixed by 
https://issues.apache.org/jira/browse/KAFKA-717 ?

> scala 2.10 build support
> 
>
> Key: KAFKA-717
> URL: https://issues.apache.org/jira/browse/KAFKA-717
> Project: Kafka
>  Issue Type: Improvement
>  Components: packaging
>Affects Versions: 0.8
>Reporter: Viktor Taranenko
>  Labels: build
> Attachments: 0001-KAFKA-717-Convert-to-scala-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 0001-common-changes-for-2.10.patch, 
> 0002-java-conversions-changes.patch, 0002-java-conversions-changes.patch, 
> 0003-add-2.9.3.patch, 0003-add-2.9.3.patch, 
> 0004-Fix-cross-compile-of-tests-update-to-2.10.2-and-set-.patch, 
> KAFKA-717-complex.patch, KAFKA-717-patches-20130720.tgz, 
> KAFKA-717-simple.patch, kafka_scala_2.10.tar.gz
>
>




--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Comment Edited] (KAFKA-717) scala 2.10 build support

2013-11-28 Thread Martin Eigenbrodt (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13835183#comment-13835183
 ] 

Martin Eigenbrodt edited comment on KAFKA-717 at 11/29/13 5:16 AM:
---

Isn't this duplicated and fixed by 
https://issues.apache.org/jira/browse/KAFKA-1046 ?


was (Author: martineigenbr...@googlemail.com):
Isn't this duplicated and fixed by 
https://issues.apache.org/jira/browse/KAFKA-717 ?

> scala 2.10 build support
> 
>
> Key: KAFKA-717
> URL: https://issues.apache.org/jira/browse/KAFKA-717
> Project: Kafka
>  Issue Type: Improvement
>  Components: packaging
>Affects Versions: 0.8
>Reporter: Viktor Taranenko
>  Labels: build
> Attachments: 0001-KAFKA-717-Convert-to-scala-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 0001-common-changes-for-2.10.patch, 
> 0002-java-conversions-changes.patch, 0002-java-conversions-changes.patch, 
> 0003-add-2.9.3.patch, 0003-add-2.9.3.patch, 
> 0004-Fix-cross-compile-of-tests-update-to-2.10.2-and-set-.patch, 
> KAFKA-717-complex.patch, KAFKA-717-patches-20130720.tgz, 
> KAFKA-717-simple.patch, kafka_scala_2.10.tar.gz
>
>




--
This message was sent by Atlassian JIRA
(v6.1#6144)


Review Request 15915: Patch for KAFKA-1152

2013-11-28 Thread Swapnil Ghike

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

Review request for kafka.


Bugs: KAFKA-1152
https://issues.apache.org/jira/browse/KAFKA-1152


Repository: kafka


Description
---

ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
leader == -1


ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
leader == -1


Diffs
-

  core/src/main/scala/kafka/server/ReplicaManager.scala 
161f58134f20f9335dbd2bee6ac3f71897cbef7c 

Diff: https://reviews.apache.org/r/15915/diff/


Testing
---


Thanks,

Swapnil Ghike



[jira] [Updated] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Swapnil Ghike updated KAFKA-1152:
-

Attachment: KAFKA-1152.patch

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152.patch, 
> KAFKA-1152_2013-11-28_10:19:05.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13835198#comment-13835198
 ] 

Swapnil Ghike commented on KAFKA-1152:
--

Created reviewboard https://reviews.apache.org/r/15915/
 against branch origin/trunk

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152.patch, 
> KAFKA-1152_2013-11-28_10:19:05.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


Re: Review Request 15901: Patch for KAFKA-1152

2013-11-28 Thread Swapnil Ghike

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

(Updated Nov. 29, 2013, 6:41 a.m.)


Review request for kafka.


Bugs: KAFKA-1152
https://issues.apache.org/jira/browse/KAFKA-1152


Repository: kafka


Description (updated)
---

ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
leader == -1


ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
leader == -1


Diffs (updated)
-

  core/src/main/scala/kafka/server/ReplicaManager.scala 
161f58134f20f9335dbd2bee6ac3f71897cbef7c 

Diff: https://reviews.apache.org/r/15901/diff/


Testing
---

Builds with all scala versions; unit tests pass


Thanks,

Swapnil Ghike



[jira] [Commented] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13835205#comment-13835205
 ] 

Swapnil Ghike commented on KAFKA-1152:
--

Updated reviewboard https://reviews.apache.org/r/15901/
 against branch origin/trunk

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152_2013-11-28_10:19:05.patch, 
> KAFKA-1152_2013-11-28_22:40:55.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Swapnil Ghike updated KAFKA-1152:
-

Attachment: KAFKA-1152_2013-11-28_22:40:55.patch

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152_2013-11-28_10:19:05.patch, 
> KAFKA-1152_2013-11-28_22:40:55.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (KAFKA-1152) ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle leader == -1

2013-11-28 Thread Swapnil Ghike (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-1152?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Swapnil Ghike updated KAFKA-1152:
-

Attachment: (was: KAFKA-1152.patch)

> ReplicaManager's handling of the leaderAndIsrRequest should gracefully handle 
> leader == -1
> --
>
> Key: KAFKA-1152
> URL: https://issues.apache.org/jira/browse/KAFKA-1152
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 0.8
>Reporter: Swapnil Ghike
>Assignee: Swapnil Ghike
> Fix For: 0.8.1
>
> Attachments: KAFKA-1152.patch, KAFKA-1152_2013-11-28_10:19:05.patch, 
> KAFKA-1152_2013-11-28_22:40:55.patch
>
>
> If a partition is created with replication factor 1, then the controller can 
> set the partition's leader to -1 in leaderAndIsrRequest when the only replica 
> of the partition is being bounced. 
> The handling of this request with a leader == -1 throws an exception on the 
> ReplicaManager which prevents the addition of fetchers for the remaining 
> partitions in the leaderAndIsrRequest.
> After the replica is bounced, the replica first receives a 
> leaderAndIsrRequest with leader == -1, then it receives another 
> leaderAndIsrRequest with the correct leader (which is the replica itself) due 
> to OfflinePartition to OnlinePartition state change. 
> In handling the first request, ReplicaManager should ignore the partition for 
> which the request has leader == -1, and continue addition of fetchers for the 
> remaining partitions. The next leaderAndIsrRequest will take care of setting 
> the correct leader for that partition.



--
This message was sent by Atlassian JIRA
(v6.1#6144)