[jira] [Created] (KAFKA-978) kafka pom file has 2 entries for zookeeper (one with exclusion, one without)

2013-07-16 Thread Jason Rosenberg (JIRA)
Jason Rosenberg created KAFKA-978:
-

 Summary: kafka pom file has 2 entries for zookeeper (one with 
exclusion, one without)
 Key: KAFKA-978
 URL: https://issues.apache.org/jira/browse/KAFKA-978
 Project: Kafka
  Issue Type: Bug
Reporter: Jason Rosenberg


I've noticed, in the 0.8-beta the pom file, there are 2 dependencies listed for 
zookeeper, one of which has an exclusion clause, the other does not.   I assume 
this is not intended:


org.apache.zookeeper
zookeeper
3.3.4


log4j
log4j


jline
jline




and then


org.apache.zookeeper
zookeeper
3.3.4


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (KAFKA-957) MirrorMaker needs to preserve the key in the source cluster

2013-07-16 Thread Guozhang Wang (JIRA)

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

Guozhang Wang updated KAFKA-957:


Attachment: KAFKA-957.v3.patch

Thanks for the comment.

For the first comment, since I needed to make a var for creating the config 
object if I use if () instead of match, I did not incorporate that. For the 
second match I have changed it to "if".

> MirrorMaker needs to preserve the key in the source cluster
> ---
>
> Key: KAFKA-957
> URL: https://issues.apache.org/jira/browse/KAFKA-957
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.8
>Reporter: Jun Rao
>Assignee: Guozhang Wang
> Attachments: KAFKA-957.v1.patch, KAFKA-957.v2.patch, 
> KAFKA-957.v2.patch, KAFKA-957.v3.patch
>
>
> Currently, MirrorMaker only propagates the message to the target cluster, but 
> not the associated key.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-967) Use key range in ProducerPerformance

2013-07-16 Thread Joel Koshy (JIRA)

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

Joel Koshy commented on KAFKA-967:
--

+1 - thanks for the patch.

> Use key range in ProducerPerformance
> 
>
> Key: KAFKA-967
> URL: https://issues.apache.org/jira/browse/KAFKA-967
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Guozhang Wang
>Assignee: Guozhang Wang
> Attachments: KAFKA-967.v1.patch, KAFKA-967.v2.patch
>
>
> Currently in ProducerPerformance, the key of the message is set to MessageID. 
> It would better to set it to a specific key within a key range (Integer type) 
> so that we can test the semantic partitioning case. This is related to 
> KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-957) MirrorMaker needs to preserve the key in the source cluster

2013-07-16 Thread Joel Koshy (JIRA)

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

Joel Koshy commented on KAFKA-957:
--

Thanks for incorporating 5 and 6. Couple additional comments:
- For the two match statements you have it is probably sufficient and
  clearer to just use if (key == null)  and if (props.contains(..))
- I'm not so sure if the trace is required but it could be useful. Would
  prefer the following format: "Sending message with key " - no need to
  show the payload. Also, may want to use java.util.Arrays.toString on the
  byte array.
- Per Jay's offline comments, hashCode in general is a bit unsafe to "rely".
  For e.g., it could be a non-uniform distribution or the underlying
  function could change. That said, your usage is safe. Still, it should be
  straightforward to do a custom hash function that we can rely on for
  consistency.


> MirrorMaker needs to preserve the key in the source cluster
> ---
>
> Key: KAFKA-957
> URL: https://issues.apache.org/jira/browse/KAFKA-957
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.8
>Reporter: Jun Rao
>Assignee: Guozhang Wang
> Attachments: KAFKA-957.v1.patch, KAFKA-957.v2.patch, 
> KAFKA-957.v2.patch
>
>
> Currently, MirrorMaker only propagates the message to the target cluster, but 
> not the associated key.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (KAFKA-957) MirrorMaker needs to preserve the key in the source cluster

2013-07-16 Thread Guozhang Wang (JIRA)

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

Guozhang Wang updated KAFKA-957:


Attachment: KAFKA-957.v2.patch

> MirrorMaker needs to preserve the key in the source cluster
> ---
>
> Key: KAFKA-957
> URL: https://issues.apache.org/jira/browse/KAFKA-957
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.8
>Reporter: Jun Rao
>Assignee: Guozhang Wang
> Attachments: KAFKA-957.v1.patch, KAFKA-957.v2.patch, 
> KAFKA-957.v2.patch
>
>
> Currently, MirrorMaker only propagates the message to the target cluster, but 
> not the associated key.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (KAFKA-957) MirrorMaker needs to preserve the key in the source cluster

2013-07-16 Thread Guozhang Wang (JIRA)

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

Guozhang Wang updated KAFKA-957:


Attachment: KAFKA-957.v2.patch

5. Use java.util.Arrays.hashCode to determine which producer to send a keyed 
message

6. Check if the partitioner class is specified instead of always trying to 
override the partitioner.

Passed testcase 5007 referring to KAFKA-976.

> MirrorMaker needs to preserve the key in the source cluster
> ---
>
> Key: KAFKA-957
> URL: https://issues.apache.org/jira/browse/KAFKA-957
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.8
>Reporter: Jun Rao
>Assignee: Guozhang Wang
> Attachments: KAFKA-957.v1.patch, KAFKA-957.v2.patch
>
>
> Currently, MirrorMaker only propagates the message to the target cluster, but 
> not the associated key.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-973) Messages From Producer Not being Partitioned

2013-07-16 Thread Subbu Srinivasan (JIRA)

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

Subbu Srinivasan commented on KAFKA-973:


Thanks Joel. I found the issue. I was instantiating a producer for every single 
send and that causes this behaviour.

(new KafkaProducer()).sendMessage(UUID.randomUUID().toString()); 






> Messages From Producer Not being Partitioned 
> -
>
> Key: KAFKA-973
> URL: https://issues.apache.org/jira/browse/KAFKA-973
> Project: Kafka
>  Issue Type: Bug
>  Components: consumer
>Affects Versions: 0.8
> Environment: Linux
>Reporter: Subbu Srinivasan
>Assignee: Neha Narkhede
>  Labels: newbie
>
> I created a two node cluster.
> 2 zoo keepers
> 2 brokers
> 1 topic with replication factor (2) and no of partition 2.
> my consumer group has two threads
> 1) From my Java client - I send few  messages to the topic. I have set 
> multiple brokers
> kafka2:9092,kafka1:9092.
> Only one thread in my consumer always gets the messages. It looks like 
> producer is not
> partitioning the requests properly.
> 2) However if I send some sample using the simple console producer, I see 
> multiple threads getting
> requests and is load balanced.
> What am I doing wrong in my client?
> public class KafkaProducer {
> 
> private final Properties props = new Properties();
> private static AtomicLong counter = new AtomicLong(0);
> kafka.javaapi.producer.Producer producer = null;
> 
> public KafkaProducer() 
> {
>   props.put("serializer.class", "kafka.serializer.StringEncoder");
>   props.put("metadata.broker.list", 
> ConfigurationUtility.getKafkaHost());
>   producer = new kafka.javaapi.producer.Producer(new 
> ProducerConfig(props));
> } 
> 
> public void sendMessage(String msg) throws Exception
> {
> producer.send(new KeyedMessage String>(ConfigurationUtility.getTopicName(), msg));
> }   
> 
> 
> public static void main(String arg[]) throws Exception
> {
> 
> ConfigurationUtility.setKafkaHost("kafka2:9092,kafka1:9092");
>   ConfigurationUtility.setTopicName("dnslog");
>   
> ConfigurationUtility.setZooKeeperHost("kafka1:2181,kafka2:2181");
>   ConfigurationUtility.setConsumerGroupId("dnslog");
>   
>   for(int i = 0 ; i < 2 ; ++i)
>   {
>   (new 
> KafkaProducer()).sendMessage(UUID.randomUUID().toString());
>   }
> }
> }

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-976) Order-Preserving Mirror Maker Testcase

2013-07-16 Thread Guozhang Wang (JIRA)

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

Guozhang Wang commented on KAFKA-976:
-

+1, passed test with the above steps.

> Order-Preserving Mirror Maker Testcase
> --
>
> Key: KAFKA-976
> URL: https://issues.apache.org/jira/browse/KAFKA-976
> Project: Kafka
>  Issue Type: Test
>Reporter: Guozhang Wang
>Assignee: John Fung
> Attachments: kafka-976-v1.patch
>
>
> A new testcase (5007) for mirror_maker_testsuite is needed for the 
> key-dependent order-preserving mirror maker, this is related to KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2013-07-16 Thread Scott Carey (JIRA)

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

Scott Carey commented on KAFKA-717:
---

The latter plan looks good to me.  

> 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-common-changes-for-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 
> 0001-KAFKA-717-Convert-to-scala-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-simple.patch, kafka_scala_2.10.tar.gz
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-976) Order-Preserving Mirror Maker Testcase

2013-07-16 Thread John Fung (JIRA)

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

John Fung commented on KAFKA-976:
-

Uploaded kafka-976-v1.patch to add testcase_5007. Please do the following to 
run the test case:

1. Check out latest 0.8 branch
2. Apply KAFKA-957.v1.patch
3. Apply KAFKA-967.v2.patch
4. Apply kafka-976-v1.patch
5. Build kafka
6. Edit /config/log4j.properties to uncomment the following 2 lines:
#log4j.logger.kafka.perf=DEBUG, kafkaAppender
#log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, 
kafkaAppender
7. Edit /system_test/testcase_to_run.json to the following:
{
"MirrorMakerTest"   : [
"testcase_5007"
]
}

8. Execute the test under /system_test :

$ python -u -B system_test_runner.py 2>&1 | tee system_test_output.log

> Order-Preserving Mirror Maker Testcase
> --
>
> Key: KAFKA-976
> URL: https://issues.apache.org/jira/browse/KAFKA-976
> Project: Kafka
>  Issue Type: Test
>Reporter: Guozhang Wang
>Assignee: John Fung
> Attachments: kafka-976-v1.patch
>
>
> A new testcase (5007) for mirror_maker_testsuite is needed for the 
> key-dependent order-preserving mirror maker, this is related to KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (KAFKA-976) Order-Preserving Mirror Maker Testcase

2013-07-16 Thread John Fung (JIRA)

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

John Fung updated KAFKA-976:


Status: Patch Available  (was: Open)

> Order-Preserving Mirror Maker Testcase
> --
>
> Key: KAFKA-976
> URL: https://issues.apache.org/jira/browse/KAFKA-976
> Project: Kafka
>  Issue Type: Test
>Reporter: Guozhang Wang
>Assignee: John Fung
> Attachments: kafka-976-v1.patch
>
>
> A new testcase (5007) for mirror_maker_testsuite is needed for the 
> key-dependent order-preserving mirror maker, this is related to KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (KAFKA-976) Order-Preserving Mirror Maker Testcase

2013-07-16 Thread John Fung (JIRA)

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

John Fung updated KAFKA-976:


Attachment: kafka-976-v1.patch

> Order-Preserving Mirror Maker Testcase
> --
>
> Key: KAFKA-976
> URL: https://issues.apache.org/jira/browse/KAFKA-976
> Project: Kafka
>  Issue Type: Test
>Reporter: Guozhang Wang
>Assignee: John Fung
> Attachments: kafka-976-v1.patch
>
>
> A new testcase (5007) for mirror_maker_testsuite is needed for the 
> key-dependent order-preserving mirror maker, this is related to KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (KAFKA-977) Implement generation/term per leader to reconcile messages correctly

2013-07-16 Thread Sriram Subramanian (JIRA)
Sriram Subramanian created KAFKA-977:


 Summary: Implement generation/term per leader to reconcile 
messages correctly
 Key: KAFKA-977
 URL: https://issues.apache.org/jira/browse/KAFKA-977
 Project: Kafka
  Issue Type: Bug
Reporter: Sriram Subramanian
Assignee: Sriram Subramanian


During unclean leader election, the log messages can diverge and when the 
followers come back up Kafka does not reconcile correctly. To implement it 
correctly, we need to add a term/generation to each message and use that to 
reconcile.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-967) Use key range in ProducerPerformance

2013-07-16 Thread Guozhang Wang (JIRA)

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

Guozhang Wang commented on KAFKA-967:
-

4. Change key type from Integer to ByteArray in the underlying producer.

> Use key range in ProducerPerformance
> 
>
> Key: KAFKA-967
> URL: https://issues.apache.org/jira/browse/KAFKA-967
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Guozhang Wang
>Assignee: Guozhang Wang
> Attachments: KAFKA-967.v1.patch, KAFKA-967.v2.patch
>
>
> Currently in ProducerPerformance, the key of the message is set to MessageID. 
> It would better to set it to a specific key within a key range (Integer type) 
> so that we can test the semantic partitioning case. This is related to 
> KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (KAFKA-976) Order-Preserving Mirror Maker Testcase

2013-07-16 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-976:
---

 Summary: Order-Preserving Mirror Maker Testcase
 Key: KAFKA-976
 URL: https://issues.apache.org/jira/browse/KAFKA-976
 Project: Kafka
  Issue Type: Test
Reporter: Guozhang Wang
Assignee: John Fung


A new testcase (5007) for mirror_maker_testsuite is needed for the 
key-dependent order-preserving mirror maker, this is related to KAFKA-957.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2013-07-16 Thread Jay Kreps (JIRA)

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

Jay Kreps commented on KAFKA-717:
-

Yeah I think we all want to do this, and likely it will work.

LinkedIn uses 2.8.0 and we can't upgrade our clients until all consumers are 
upgraded. This is happening now but takes a while (we have hundreds of 
applications) so we aren't going to be off 2.8.0 in the next few weeks for 0.8.

LinkedIn can do one of two things:
- Work off a local fork for linkedin temporarily for 0.8 so we can keep running 
2.8
- Wait until 0.8.1 and let people who really want 2.10 apply the patch

Personally I think the project is better off with the later strategy so that 
0.8 is getting any fixes from our production deploys, but you might disagree.

> 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-common-changes-for-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 
> 0001-KAFKA-717-Convert-to-scala-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-simple.patch, kafka_scala_2.10.tar.gz
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2013-07-16 Thread Scott Carey (JIRA)

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

Scott Carey commented on KAFKA-717:
---

{quote}that would be easiest for us would be post-0.8 (i.e. 0.8.1/trunk){quote} 
   (Can the wiki-text plugin be enabled for this project?)

That's fine,  I'm happy maintaining our own fork to handle 2.10.x for now.   
This is about how to move forward long term, which requires trimming support 
for old versions.  2.11 (a 'performance' release) will be out in 6 months or 
so, and 2.8.x will become even more difficult to support in conjunction. 

In this case, the big pain point is the Java conversions/converters code in the 
Java API.   New client APIs in Kafka can simultaneously make the code more 
portable across Scala versions, since incompatibilities are generally due to 
library changes and not language changes.



> 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-common-changes-for-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 
> 0001-KAFKA-717-Convert-to-scala-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-simple.patch, kafka_scala_2.10.tar.gz
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2013-07-16 Thread Scott Carey (JIRA)

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

Scott Carey commented on KAFKA-717:
---

{quote}As it stands now folks are already running 0.8.0-beta1 in 
production{quote}

We have been up and running in production with 0.8 before beta-1, with Scala 
2.10.1. About a month ago we moved to the most recent 0.8 with Scala 2.10.2.  
2.10.x seems to provide a minor performance boost as well (faster Scala 
collections).
Our load is not as high as LI and some others, but it is 250GB /day (after 
compression), involves multiple partition topics, and heavily leverages the 
mirroring.   The performance was worse than 0.7.x at first, but it is catching 
back up.

Compiling with 2.10.x actually highlights a LOT of warnings that could indicate 
bugs in Kafka that 2.8.x compiler versions do not display.  I am more worried 
about those, possible bugs, than what upgrading to a newer Scala version brings.

I am much more concerned with the need for cross-compile for clients than for 
the broker.  The broker is isolated and can be tested on its own and default to 
a newer Scala version.  Consumer or Producer client libraries however have to 
live in another application, and that application may have other dependencies 
that constrain what version is appropriate.


> 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-common-changes-for-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 
> 0001-KAFKA-717-Convert-to-scala-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-simple.patch, kafka_scala_2.10.tar.gz
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2013-07-16 Thread Jay Kreps (JIRA)

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

Jay Kreps commented on KAFKA-717:
-

We are up for switching over at LinkedIn, and will let people know how it goes. 
We have actually been delaying to get the 0.8 release done in hope of changing 
one thing at a time. The timeframe that would be easiest for us would be 
post-0.8 (i.e. 0.8.1/trunk).

> 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-common-changes-for-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 
> 0001-KAFKA-717-Convert-to-scala-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-simple.patch, kafka_scala_2.10.tar.gz
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (KAFKA-975) Leader not local for partition when partition is leader

2013-07-16 Thread Dan Swanson (JIRA)
Dan Swanson created KAFKA-975:
-

 Summary: Leader not local for partition when partition is leader
 Key: KAFKA-975
 URL: https://issues.apache.org/jira/browse/KAFKA-975
 Project: Kafka
  Issue Type: Bug
  Components: replication
Affects Versions: 0.8
 Environment: centos 6.4
Reporter: Dan Swanson
Assignee: Neha Narkhede


I have a two server kafka cluster (dev003 and dev004).  I am following the 
example from this URL but using two servers with a single kafka instance 
instead of using 1 server with two instances..

http://www.michael-noll.com/blog/2013/03/13/running-a-multi-broker-apache-kafka-cluster-on-a-single-node/

Using the following trunk version

commit c27c768463a5dc6be113f2e5b3e00bf8d9d9d602
Author: David Arthur 
Date:   Thu Jul 11 15:34:57 2013 -0700

KAFKA-852, remove clientId from Offset{Fetch,Commit}Response. Reviewed by 
Jay.

--

[2013-07-16 10:56:50,279] INFO [Kafka Server 3], started 
(kafka.server.KafkaServer)

--

dan@linux-rr29:~/git-data/kafka-current-src> bin/kafka-topics.sh --zookeeper 
dev003:2181 --create --topic dadj1 --partitions 1 --replication-factor 2 
2>/dev/null
Created topic "dadj1".
dan@linux-rr29:~/git-data/kafka-current-src>

---


[2013-07-16 10:56:57,946] INFO [Replica Manager on Broker 3]: Handling 
LeaderAndIsr request 
Name:LeaderAndIsrRequest;Version:0;Controller:4;ControllerEpoch:19;CorrelationId:12;ClientId:id_4-host_dev004-port_9092;PartitionState:(dadj1,0)
 -> 
(LeaderAndIsrInfo:(Leader:3,ISR:3,4,LeaderEpoch:0,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:3,4);Leaders:id:3,host:dev003,port:9092
 (kafka.server.ReplicaManager)
[2013-07-16 10:56:57,959] INFO [ReplicaFetcherManager on broker 3] Removing 
fetcher for partition [dadj1,0] (kafka.server.ReplicaFetcherManager)
[2013-07-16 10:57:21,196] WARN [KafkaApi-3] Produce request with correlation id 
2 from client  on partition [dadj1,0] failed due to Leader not local for 
partition [dadj1,0] on broker 3 (kafka.server.KafkaApis)

-

dan@linux-rr29:~/git-data/kafka-current-src> bin/kafka-topics.sh --zookeeper 
dev003:2181 --describe --topic dadj1 2>/dev/null
dadj1
configs: 
partitions: 1
topic: dadj1partition: 0leader: 3   replicas: 3,4   
isr: 3,4
dan@linux-rr29:~/git-data/kafka-current-src>

Dev003 logs show that server is elected as leader and has correct id of 3, 
zookeeper shows dev003 is leader, but when I try to produce to the topic I get 
a failure because the server thinks it is not the leader.  This occurs 
regardless of which server (dev003 or dev004) ends up the leader.

Here is my config which is the same except for the broker id and host names

[root@dev003 kafka-current-src]# grep -v -e '^#' -e '^$' 
config/server.properties 
broker.id=3
port=9092
host.name=dev003
num.network.threads=2
 
num.io.threads=2
socket.send.buffer.bytes=1048576
socket.receive.buffer.bytes=1048576
socket.request.max.bytes=104857600
log.dir=/opt/kafka/data/8.0/
num.partitions=1
log.flush.interval.messages=1
log.flush.interval.ms=1000
log.retention.hours=168
log.segment.bytes=536870912
log.cleanup.interval.mins=1
zookeeper.connect=10.200.8.61:2181,10.200.8.62:2181,10.200.8.63:2181
zookeeper.connection.timeout.ms=100
kafka.metrics.polling.interval.secs=5
kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter
kafka.csv.metrics.dir=/tmp/kafka_metrics
kafka.csv.metrics.reporter.enabled=false
[root@dev003 kafka-current-src]#



--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (KAFKA-975) Leader not local for partition when partition is leader (kafka.common.NotLeaderForPartitionException)

2013-07-16 Thread Dan Swanson (JIRA)

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

Dan Swanson commented on KAFKA-975:
---

I have also tried using IP instead of host names in the config but that does 
not seem to make a difference

> Leader not local for partition when partition is leader 
> (kafka.common.NotLeaderForPartitionException)
> -
>
> Key: KAFKA-975
> URL: https://issues.apache.org/jira/browse/KAFKA-975
> Project: Kafka
>  Issue Type: Bug
>  Components: replication
>Affects Versions: 0.8
> Environment: centos 6.4
>Reporter: Dan Swanson
>Assignee: Neha Narkhede
>
> I have a two server kafka cluster (dev003 and dev004).  I am following the 
> example from this URL but using two servers with a single kafka instance 
> instead of using 1 server with two instances..
> http://www.michael-noll.com/blog/2013/03/13/running-a-multi-broker-apache-kafka-cluster-on-a-single-node/
> Using the following trunk version
> commit c27c768463a5dc6be113f2e5b3e00bf8d9d9d602
> Author: David Arthur 
> Date:   Thu Jul 11 15:34:57 2013 -0700
> KAFKA-852, remove clientId from Offset{Fetch,Commit}Response. Reviewed by 
> Jay.
> --
> [2013-07-16 10:56:50,279] INFO [Kafka Server 3], started 
> (kafka.server.KafkaServer)
> --
> dan@linux-rr29:~/git-data/kafka-current-src> bin/kafka-topics.sh --zookeeper 
> dev003:2181 --create --topic dadj1 --partitions 1 --replication-factor 2 
> 2>/dev/null
> Created topic "dadj1".
> dan@linux-rr29:~/git-data/kafka-current-src>
> ---
> [2013-07-16 10:56:57,946] INFO [Replica Manager on Broker 3]: Handling 
> LeaderAndIsr request 
> Name:LeaderAndIsrRequest;Version:0;Controller:4;ControllerEpoch:19;CorrelationId:12;ClientId:id_4-host_dev004-port_9092;PartitionState:(dadj1,0)
>  -> 
> (LeaderAndIsrInfo:(Leader:3,ISR:3,4,LeaderEpoch:0,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:3,4);Leaders:id:3,host:dev003,port:9092
>  (kafka.server.ReplicaManager)
> [2013-07-16 10:56:57,959] INFO [ReplicaFetcherManager on broker 3] Removing 
> fetcher for partition [dadj1,0] (kafka.server.ReplicaFetcherManager)
> [2013-07-16 10:57:21,196] WARN [KafkaApi-3] Produce request with correlation 
> id 2 from client  on partition [dadj1,0] failed due to Leader not local for 
> partition [dadj1,0] on broker 3 (kafka.server.KafkaApis)
> -
> dan@linux-rr29:~/git-data/kafka-current-src> bin/kafka-topics.sh --zookeeper 
> dev003:2181 --describe --topic dadj1 2>/dev/null
> dadj1
>   configs: 
>   partitions: 1
>   topic: dadj1partition: 0leader: 3   replicas: 3,4   
> isr: 3,4
> dan@linux-rr29:~/git-data/kafka-current-src>
> Dev003 logs show that server is elected as leader and has correct id of 3, 
> zookeeper shows dev003 is leader, but when I try to produce to the topic I 
> get a failure because the server thinks it is not the leader.  This occurs 
> regardless of which server (dev003 or dev004) ends up the leader.
> Here is my config which is the same except for the broker id and host names
> [root@dev003 kafka-current-src]# grep -v -e '^#' -e '^$' 
> config/server.properties 
> broker.id=3
> port=9092
> host.name=dev003
> num.network.threads=2
>  
> num.io.threads=2
> socket.send.buffer.bytes=1048576
> socket.receive.buffer.bytes=1048576
> socket.request.max.bytes=104857600
> log.dir=/opt/kafka/data/8.0/
> num.partitions=1
> log.flush.interval.messages=1
> log.flush.interval.ms=1000
> log.retention.hours=168
> log.segment.bytes=536870912
> log.cleanup.interval.mins=1
> zookeeper.connect=10.200.8.61:2181,10.200.8.62:2181,10.200.8.63:2181
> zookeeper.connection.timeout.ms=100
> kafka.metrics.polling.interval.secs=5
> kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter
> kafka.csv.metrics.dir=/tmp/kafka_metrics
> kafka.csv.metrics.reporter.enabled=false
> [root@dev003 kafka-current-src]#

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (KAFKA-975) Leader not local for partition when partition is leader (kafka.common.NotLeaderForPartitionException)

2013-07-16 Thread Dan Swanson (JIRA)

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

Dan Swanson updated KAFKA-975:
--

Summary: Leader not local for partition when partition is leader 
(kafka.common.NotLeaderForPartitionException)  (was: Leader not local for 
partition when partition is leader)

> Leader not local for partition when partition is leader 
> (kafka.common.NotLeaderForPartitionException)
> -
>
> Key: KAFKA-975
> URL: https://issues.apache.org/jira/browse/KAFKA-975
> Project: Kafka
>  Issue Type: Bug
>  Components: replication
>Affects Versions: 0.8
> Environment: centos 6.4
>Reporter: Dan Swanson
>Assignee: Neha Narkhede
>
> I have a two server kafka cluster (dev003 and dev004).  I am following the 
> example from this URL but using two servers with a single kafka instance 
> instead of using 1 server with two instances..
> http://www.michael-noll.com/blog/2013/03/13/running-a-multi-broker-apache-kafka-cluster-on-a-single-node/
> Using the following trunk version
> commit c27c768463a5dc6be113f2e5b3e00bf8d9d9d602
> Author: David Arthur 
> Date:   Thu Jul 11 15:34:57 2013 -0700
> KAFKA-852, remove clientId from Offset{Fetch,Commit}Response. Reviewed by 
> Jay.
> --
> [2013-07-16 10:56:50,279] INFO [Kafka Server 3], started 
> (kafka.server.KafkaServer)
> --
> dan@linux-rr29:~/git-data/kafka-current-src> bin/kafka-topics.sh --zookeeper 
> dev003:2181 --create --topic dadj1 --partitions 1 --replication-factor 2 
> 2>/dev/null
> Created topic "dadj1".
> dan@linux-rr29:~/git-data/kafka-current-src>
> ---
> [2013-07-16 10:56:57,946] INFO [Replica Manager on Broker 3]: Handling 
> LeaderAndIsr request 
> Name:LeaderAndIsrRequest;Version:0;Controller:4;ControllerEpoch:19;CorrelationId:12;ClientId:id_4-host_dev004-port_9092;PartitionState:(dadj1,0)
>  -> 
> (LeaderAndIsrInfo:(Leader:3,ISR:3,4,LeaderEpoch:0,ControllerEpoch:19),ReplicationFactor:2),AllReplicas:3,4);Leaders:id:3,host:dev003,port:9092
>  (kafka.server.ReplicaManager)
> [2013-07-16 10:56:57,959] INFO [ReplicaFetcherManager on broker 3] Removing 
> fetcher for partition [dadj1,0] (kafka.server.ReplicaFetcherManager)
> [2013-07-16 10:57:21,196] WARN [KafkaApi-3] Produce request with correlation 
> id 2 from client  on partition [dadj1,0] failed due to Leader not local for 
> partition [dadj1,0] on broker 3 (kafka.server.KafkaApis)
> -
> dan@linux-rr29:~/git-data/kafka-current-src> bin/kafka-topics.sh --zookeeper 
> dev003:2181 --describe --topic dadj1 2>/dev/null
> dadj1
>   configs: 
>   partitions: 1
>   topic: dadj1partition: 0leader: 3   replicas: 3,4   
> isr: 3,4
> dan@linux-rr29:~/git-data/kafka-current-src>
> Dev003 logs show that server is elected as leader and has correct id of 3, 
> zookeeper shows dev003 is leader, but when I try to produce to the topic I 
> get a failure because the server thinks it is not the leader.  This occurs 
> regardless of which server (dev003 or dev004) ends up the leader.
> Here is my config which is the same except for the broker id and host names
> [root@dev003 kafka-current-src]# grep -v -e '^#' -e '^$' 
> config/server.properties 
> broker.id=3
> port=9092
> host.name=dev003
> num.network.threads=2
>  
> num.io.threads=2
> socket.send.buffer.bytes=1048576
> socket.receive.buffer.bytes=1048576
> socket.request.max.bytes=104857600
> log.dir=/opt/kafka/data/8.0/
> num.partitions=1
> log.flush.interval.messages=1
> log.flush.interval.ms=1000
> log.retention.hours=168
> log.segment.bytes=536870912
> log.cleanup.interval.mins=1
> zookeeper.connect=10.200.8.61:2181,10.200.8.62:2181,10.200.8.63:2181
> zookeeper.connection.timeout.ms=100
> kafka.metrics.polling.interval.secs=5
> kafka.metrics.reporters=kafka.metrics.KafkaCSVMetricsReporter
> kafka.csv.metrics.dir=/tmp/kafka_metrics
> kafka.csv.metrics.reporter.enabled=false
> [root@dev003 kafka-current-src]#

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


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

2013-07-16 Thread Joe Stein (JIRA)

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

Joe Stein commented on KAFKA-717:
-

Moving away from 2.8.X for the broker build would be more feasible when folks 
running brokers in production are running them on 2.9.X or even 2.10.X and 
right now I don't know of anyone running a broker not on 2.8.X with any real 
volume (but that does not mean its not happening just I don't know about it).  
I will work over the next few weeks to start to pull more of this information 
together so its empirical and not assumed.

As it stands now folks are already running 0.8.0-beta1 in production and there 
is only a certain amount of beta that production operation folks will tolerate 
and upgrading from Scala 2.8.0 to another version for companies that have 
terabytes a day going through Kafka as a key part of their operations may not 
be feasible just yet.

While I agree with you 100% I also understand and appreciate how we need to 
support this project (lots of folks to support in different ways) and should be 
supporting this project for the stability it offers and rushing ahead has to be 
tempered with production support by the Apache Kafka community and PMC.

>From my perspective the scalaVersion default is for the broker and the cross 
>compiles are for different needs of producer/consumer (at least how I see it) 
>and now that we are publishing to maven the default should matter even more 
>only for the broker (again IMHO)

That all being said I looked through and get the patches now and will give 
these a shot in order using the 06/14/2013 dates

0001-common-changes-for-2.10.patch
0002-java-conversions-changes.patch
0003-add-2.9.3.patch
0004-Fix-cross-compile-of-tests-update-to-2.10.2-and-set-.patch

I also have some ideas about ways to making this available sooner for some 
things I am working on the next few weeks moving forward.

Thank you for the patches and patience.  The best is yet to come!


> 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-common-changes-for-2.10.patch, 
> 0001-common-changes-for-2.10.patch, 
> 0001-KAFKA-717-Convert-to-scala-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-simple.patch, kafka_scala_2.10.tar.gz
>
>


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira