[jira] [Commented] (KAFKA-990) Fix ReassignPartitionCommand and improve usability

2013-08-08 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13733796#comment-13733796
 ] 

Neha Narkhede commented on KAFKA-990:
-

Thanks for the rebased patch, Sriram. Overall, the changes look great. +1. One 
minor suggestion -

ReassignPartitionsCommand

For determining the replication factor for replica assignment, can we just use 
the first or last partition in the map instead of relying on a partition id 0? 
That way if we change the assumption that partition id should always start from 
0, this will not break. -
topicInfo._2.head._2.size instead of 
topicInfo._2.get(TopicAndPartition(topicInfo._1, 0)).get.size

Also, it seems that #2 in the description above was not really a problem. This 
is because onPartitionReassignment checks areReplicasInIsr and hence restarts 
the reassignment correctly. This is however not true if we hit #1, which is a 
real issue.

 Fix ReassignPartitionCommand and improve usability
 --

 Key: KAFKA-990
 URL: https://issues.apache.org/jira/browse/KAFKA-990
 Project: Kafka
  Issue Type: Bug
Reporter: Sriram Subramanian
Assignee: Sriram Subramanian
 Attachments: KAFKA-990-v1.patch, KAFKA-990-v1-rebased.patch


 1. The tool does not register for IsrChangeListener on controller failover.
 2. There is a race condition where the previous listener can fire on 
 controller failover and the replicas can be in ISR. Even after re-registering 
 the ISR listener after failover, it will never be triggered.
 3. The input the tool is a static list which is very hard to use. To improve 
 this, as a first step the tool needs to take a list of topics and list of 
 brokers to do the assignment to and then generate the reassignment plan.

--
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] [Comment Edited] (KAFKA-990) Fix ReassignPartitionCommand and improve usability

2013-08-08 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13733796#comment-13733796
 ] 

Neha Narkhede edited comment on KAFKA-990 at 8/8/13 6:16 PM:
-

Thanks for the rebased patch, Sriram. Overall, the changes look great. +1. One 
minor suggestion -

ReassignPartitionsCommand

For determining the replication factor for replica assignment, can we just use 
the first or last partition in the map instead of relying on a partition id 0? 
That way if we change the assumption that partition id should always start from 
0, this will not break. -
topicInfo._2.head._2.size instead of 
topicInfo._2.get(TopicAndPartition(topicInfo._1, 0)).get.size

If you are ok with this suggestion, I can make it on checkin.

Also, it seems that #2 in the description above was not really a problem. This 
is because onPartitionReassignment checks areReplicasInIsr and hence restarts 
the reassignment correctly. This is however not true if we hit #1, which is a 
real issue.

  was (Author: nehanarkhede):
Thanks for the rebased patch, Sriram. Overall, the changes look great. +1. 
One minor suggestion -

ReassignPartitionsCommand

For determining the replication factor for replica assignment, can we just use 
the first or last partition in the map instead of relying on a partition id 0? 
That way if we change the assumption that partition id should always start from 
0, this will not break. -
topicInfo._2.head._2.size instead of 
topicInfo._2.get(TopicAndPartition(topicInfo._1, 0)).get.size

Also, it seems that #2 in the description above was not really a problem. This 
is because onPartitionReassignment checks areReplicasInIsr and hence restarts 
the reassignment correctly. This is however not true if we hit #1, which is a 
real issue.
  
 Fix ReassignPartitionCommand and improve usability
 --

 Key: KAFKA-990
 URL: https://issues.apache.org/jira/browse/KAFKA-990
 Project: Kafka
  Issue Type: Bug
Reporter: Sriram Subramanian
Assignee: Sriram Subramanian
 Attachments: KAFKA-990-v1.patch, KAFKA-990-v1-rebased.patch


 1. The tool does not register for IsrChangeListener on controller failover.
 2. There is a race condition where the previous listener can fire on 
 controller failover and the replicas can be in ISR. Even after re-registering 
 the ISR listener after failover, it will never be triggered.
 3. The input the tool is a static list which is very hard to use. To improve 
 this, as a first step the tool needs to take a list of topics and list of 
 brokers to do the assignment to and then generate the reassignment plan.

--
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-990) Fix ReassignPartitionCommand and improve usability

2013-08-08 Thread Joel Koshy (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-990?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13733837#comment-13733837
 ] 

Joel Koshy commented on KAFKA-990:
--

- Topics to move json file format seems unnecessarily complicated. Why not just 
a JSON array?
- Use CommandLineUtils.checkRequiredArgs
- May be helpful to also print out the existing partition assignment and the 
final assignment.
- dryrun to dry-run which I think is the spelling unix tools like patch 
tend to use.
- line 88: use head instead of assuming 0 exists (start partition id could be 
!= 0)

I did not finish going through all the changes in controller, but thought I 
would put in my comments so far :)


 Fix ReassignPartitionCommand and improve usability
 --

 Key: KAFKA-990
 URL: https://issues.apache.org/jira/browse/KAFKA-990
 Project: Kafka
  Issue Type: Bug
Reporter: Sriram Subramanian
Assignee: Sriram Subramanian
 Attachments: KAFKA-990-v1.patch, KAFKA-990-v1-rebased.patch


 1. The tool does not register for IsrChangeListener on controller failover.
 2. There is a race condition where the previous listener can fire on 
 controller failover and the replicas can be in ISR. Even after re-registering 
 the ISR listener after failover, it will never be triggered.
 3. The input the tool is a static list which is very hard to use. To improve 
 this, as a first step the tool needs to take a list of topics and list of 
 brokers to do the assignment to and then generate the reassignment plan.

--
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] Subscription: outstanding kafka patches

2013-08-08 Thread jira
Issue Subscription
Filter: outstanding kafka patches (66 issues)
The list of outstanding kafka patches
Subscriber: kafka-mailing-list

Key Summary
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-995   Enforce that the value for replica.fetch.max.bytes is always = the 
value for message.max.bytes
https://issues.apache.org/jira/browse/KAFKA-995
KAFKA-990   Fix ReassignPartitionCommand and improve usability
https://issues.apache.org/jira/browse/KAFKA-990
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-982   Logo for Kafka
https://issues.apache.org/jira/browse/KAFKA-982
KAFKA-981   Unable to pull Kafka binaries with Ivy
https://issues.apache.org/jira/browse/KAFKA-981
KAFKA-976   Order-Preserving Mirror Maker Testcase
https://issues.apache.org/jira/browse/KAFKA-976
KAFKA-974   can't use public release maven repo because of failure of 
downloaded dependency
https://issues.apache.org/jira/browse/KAFKA-974
KAFKA-967   Use key range in ProducerPerformance
https://issues.apache.org/jira/browse/KAFKA-967
KAFKA-956   High-level consumer fails to check topic metadata response for 
errors
https://issues.apache.org/jira/browse/KAFKA-956
KAFKA-955   After a leader change, messages sent with ack=0 are lost
https://issues.apache.org/jira/browse/KAFKA-955
KAFKA-946   Kafka Hadoop Consumer fails when verifying message checksum
https://issues.apache.org/jira/browse/KAFKA-946
KAFKA-923   Improve controller failover latency
https://issues.apache.org/jira/browse/KAFKA-923
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 specific code from the 
RequestChannel
https://issues.apache.org/jira/browse/KAFKA-745
KAFKA-735   Add looping and JSON output for ConsumerOffsetChecker
https://issues.apache.org/jira/browse/KAFKA-735
KAFKA-717   scala 2.10 build support
https://issues.apache.org/jira/browse/KAFKA-717
KAFKA-686   0.8 Kafka broker should give a better error message when running 
against 0.7 zookeeper
https://issues.apache.org/jira/browse/KAFKA-686
KAFKA-677   Retention process gives exception if an empty segment is chosen for 
collection
https://issues.apache.org/jira/browse/KAFKA-677
KAFKA-674   Clean Shutdown Testing - Log segments checksums mismatch
https://issues.apache.org/jira/browse/KAFKA-674
KAFKA-652   Create testcases for clean shut-down
https://issues.apache.org/jira/browse/KAFKA-652
KAFKA-649   Cleanup log4j logging
https://issues.apache.org/jira/browse/KAFKA-649
KAFKA-645   Create a shell script to run System Test with DEBUG details and 
tee console output to a file
https://issues.apache.org/jira/browse/KAFKA-645
KAFKA-598   decouple fetch size from max message size
https://issues.apache.org/jira/browse/KAFKA-598
KAFKA-583   SimpleConsumerShell may receive less data inconsistently
https://issues.apache.org/jira/browse/KAFKA-583
KAFKA-559   Garbage collect old consumer metadata entries
https://issues.apache.org/jira/browse/KAFKA-559
KAFKA-552   No error messages logged for those failing-to-send messages from 
Producer

[jira] [Created] (KAFKA-1004) Handle topic event for trivial whitelist topic filters

2013-08-08 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-1004:


 Summary: Handle topic event for trivial whitelist topic filters
 Key: KAFKA-1004
 URL: https://issues.apache.org/jira/browse/KAFKA-1004
 Project: Kafka
  Issue Type: Bug
Reporter: Guozhang Wang
Assignee: Guozhang Wang
 Fix For: 0.8.1, 0.7


Toay consumer's TopicEventWatcher is not subscribed with trivial whitelist 
topic names. Hence if the topic is not registered on ZK when the consumer is 
started, it will not trigger the rebalance of consumers later when it is 
created and hence not be consumed even if it is in the whilelist. A proposed 
fix would be always subscribe TopicEventWatcher for all whitelist consumers.

--
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-992) Double Check on Broker Registration to Avoid False NodeExist Exception

2013-08-08 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-992?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13734114#comment-13734114
 ] 

Neha Narkhede commented on KAFKA-992:
-

Thanks for the follow up patch Guozhang. Overall, looks correct. Few minor 
suggestions -

9. ZkUtils

9.1. Could you add more details in the log message when the json parsing of the 
controller path fails? Since we know we are changing the format, something 
along the lines of Json parsing of the controller path failed. Probably this 
controller is still using the old format [%s] of storing the broker id in the 
zookeeper path
9.2 We don't need to convert the controller variable to string since it is 
already a string
9.3 Improve the error message when both json parsing and the toInt conversion 
fails. Failed to parse the leader leaderinfo  doesn't say that we failed to 
parse the controller's
 leader election path.

10. ZookeeperLeaderElector
10.1 Remove unused import BrokerNotAvailableException
10.2 In elect() API, should'nt we use readDataMaybeNull instead of readData? 
That covers the case if the ephemeral node disappears before you get a chance 
to read it.
10.3 Since the changes to elect() are new, I suggest we convert the debug to 
info or warn statements. This elect() is rarely called, this will not pollute 
the log.
10.4 One suggestion to reduce code and make it somewhat cleaner - If we change 
electFinished to electionNotDone, we need to change it only in one place - 
where we don't need to retry. Currently we have to change electFinished 
multiple times at different places


 Double Check on Broker Registration to Avoid False NodeExist Exception
 --

 Key: KAFKA-992
 URL: https://issues.apache.org/jira/browse/KAFKA-992
 Project: Kafka
  Issue Type: Bug
Reporter: Neha Narkhede
Assignee: Guozhang Wang
 Attachments: KAFKA-992.v1.patch, KAFKA-992.v2.patch, 
 KAFKA-992.v3.patch, KAFKA-992.v4.patch, KAFKA-992.v5.patch, KAFKA-992.v6.patch


 The current behavior of zookeeper for ephemeral nodes is that session 
 expiration and ephemeral node deletion is not an atomic operation. 
 The side-effect of the above zookeeper behavior in Kafka, for certain corner 
 cases, is that ephemeral nodes can be lost even if the session is not 
 expired. The sequence of events that can lead to lossy ephemeral nodes is as 
 follows -
 1. The session expires on the client, it assumes the ephemeral nodes are 
 deleted, so it establishes a new session with zookeeper and tries to 
 re-create the ephemeral nodes. 
 2. However, when it tries to re-create the ephemeral node,zookeeper throws 
 back a NodeExists error code. Now this is legitimate during a session 
 disconnect event (since zkclient automatically retries the
 operation and raises a NodeExists error). Also by design, Kafka server 
 doesn't have multiple zookeeper clients create the same ephemeral node, so 
 Kafka server assumes the NodeExists is normal. 
 3. However, after a few seconds zookeeper deletes that ephemeral node. So 
 from the client's perspective, even though the client has a new valid 
 session, its ephemeral node is gone.
 This behavior is triggered due to very long fsync operations on the zookeeper 
 leader. When the leader wakes up from such a long fsync operation, it has 
 several sessions to expire. And the time between the session expiration and 
 the ephemeral node deletion is magnified. Between these 2 operations, a 
 zookeeper client can issue a ephemeral node creation operation, that could've 
 appeared to have succeeded, but the leader later deletes the ephemeral node 
 leading to permanent ephemeral node loss from the client's perspective. 
 Thread from zookeeper mailing list: 
 http://zookeeper.markmail.org/search/?q=Zookeeper+3.3.4#query:Zookeeper%203.3.4%20date%3A201307%20+page:1+mid:zma242a2qgp6gxvx+state:results

--
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-992) Double Check on Broker Registration to Avoid False NodeExist Exception

2013-08-08 Thread Guozhang Wang (JIRA)

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

Guozhang Wang updated KAFKA-992:


Attachment: KAFKA-992.v7.patch

Thanks for the comment Neha.

9.1: Done. 
9.2,3 Done.

10.1,2,3: Done
10.4: Done. Great point!

 Double Check on Broker Registration to Avoid False NodeExist Exception
 --

 Key: KAFKA-992
 URL: https://issues.apache.org/jira/browse/KAFKA-992
 Project: Kafka
  Issue Type: Bug
Reporter: Neha Narkhede
Assignee: Guozhang Wang
 Attachments: KAFKA-992.v1.patch, KAFKA-992.v2.patch, 
 KAFKA-992.v3.patch, KAFKA-992.v4.patch, KAFKA-992.v5.patch, 
 KAFKA-992.v6.patch, KAFKA-992.v7.patch


 The current behavior of zookeeper for ephemeral nodes is that session 
 expiration and ephemeral node deletion is not an atomic operation. 
 The side-effect of the above zookeeper behavior in Kafka, for certain corner 
 cases, is that ephemeral nodes can be lost even if the session is not 
 expired. The sequence of events that can lead to lossy ephemeral nodes is as 
 follows -
 1. The session expires on the client, it assumes the ephemeral nodes are 
 deleted, so it establishes a new session with zookeeper and tries to 
 re-create the ephemeral nodes. 
 2. However, when it tries to re-create the ephemeral node,zookeeper throws 
 back a NodeExists error code. Now this is legitimate during a session 
 disconnect event (since zkclient automatically retries the
 operation and raises a NodeExists error). Also by design, Kafka server 
 doesn't have multiple zookeeper clients create the same ephemeral node, so 
 Kafka server assumes the NodeExists is normal. 
 3. However, after a few seconds zookeeper deletes that ephemeral node. So 
 from the client's perspective, even though the client has a new valid 
 session, its ephemeral node is gone.
 This behavior is triggered due to very long fsync operations on the zookeeper 
 leader. When the leader wakes up from such a long fsync operation, it has 
 several sessions to expire. And the time between the session expiration and 
 the ephemeral node deletion is magnified. Between these 2 operations, a 
 zookeeper client can issue a ephemeral node creation operation, that could've 
 appeared to have succeeded, but the leader later deletes the ephemeral node 
 leading to permanent ephemeral node loss from the client's perspective. 
 Thread from zookeeper mailing list: 
 http://zookeeper.markmail.org/search/?q=Zookeeper+3.3.4#query:Zookeeper%203.3.4%20date%3A201307%20+page:1+mid:zma242a2qgp6gxvx+state:results

--
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


Re: Kafka/Hadoop consumers and producers

2013-08-08 Thread Felix GV
The contrib code is simple and probably wouldn't require too much work to
fix, but it's a lot less robust than Camus, so you would ideally need to do
some work to make it solid against all edge cases, failure scenarios and
performance bottlenecks...

I would definitely recommend investing in Camus instead, since it already
covers a lot of the challenges I'm mentioning above, and also has more
community support behind it at the moment (as far as I can tell, anyway),
so it is more likely to keep getting improvements than the contrib code.

--
Felix


On Thu, Aug 8, 2013 at 9:28 AM, psaltis.and...@gmail.com wrote:

 We also have a need today to ETL from Kafka into Hadoop and we do not
 currently nor have any plans to use Avro.

 So is the official direction based on this discussion to ditch the Kafka
 contrib code and direct people to use Camus without Avro as Ken described
 or are both solutions going to survive?

 I can put time into the contrib code and/or work on documenting the
 tutorial on how to make Camus work without Avro.

 Which is the preferred route, for the long term?

 Thanks,
 Andrew

 On Wednesday, August 7, 2013 10:50:53 PM UTC-6, Ken Goodhope wrote:
  Hi Andrew,
 
 
 
  Camus can be made to work without avro. You will need to implement a
 message decoder and and a data writer.   We need to add a better tutorial
 on how to do this, but it isn't that difficult. If you decide to go down
 this path, you can always ask questions on this list. I try to make sure
 each email gets answered. But it can take me a day or two.
 
 
 
  -Ken
 
 
 
  On Aug 7, 2013, at 9:33 AM, ao...@wikimedia.org wrote:
 
 
 
   Hi all,
 
  
 
   Over at the Wikimedia Foundation, we're trying to figure out the best
 way to do our ETL from Kafka into Hadoop.  We don't currently use Avro and
 I'm not sure if we are going to.  I came across this post.
 
  
 
   If the plan is to remove the hadoop-consumer from Kafka contrib, do
 you think we should not consider it as one of our viable options?
 
  
 
   Thanks!
 
   -Andrew
 
  
 
   --
 
   You received this message because you are subscribed to the Google
 Groups Camus - Kafka ETL for Hadoop group.
 
   To unsubscribe from this group and stop receiving emails from it, send
 an email to camus_etl+unsubscr...@googlegroups.com.
 
   For more options, visit https://groups.google.com/groups/opt_out.
 
  
 
  




[jira] [Created] (KAFKA-1005) kafka.perf.ConsumerPerformance not shutting down consumer

2013-08-08 Thread paul mackles (JIRA)
paul mackles created KAFKA-1005:
---

 Summary: kafka.perf.ConsumerPerformance not shutting down consumer
 Key: KAFKA-1005
 URL: https://issues.apache.org/jira/browse/KAFKA-1005
 Project: Kafka
  Issue Type: Bug
  Components: tools
Reporter: paul mackles
Priority: Minor


I have been using the consumer-perf and producer-perf scripts to try out 
different failure scenarios with 0.8. In one such test I had consumer-perf 
reading from a topic that was no longer being written to. While consumer-perf 
finished normally, I noticed that ConsumerOffsetChecker reported lags 0 for 
several partitions on my topic. I believe this is due to 
kafka.perf.ConsumerPerformance not calling shutdown on the consumer after all 
of the threads have completed. After adding the shutdown call, I was able to 
verify that lag=0 for all partitions on my test topic after consumer-perf 
finished normally.

This is pretty minor but since I am guessing the perf tools are used pretty 
heavily by newbies like myself, might as well make them right.

Patch attached.

--
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


Re: Kafka/Hadoop consumers and producers

2013-08-08 Thread Andrew Psaltis
Felix,
The Camus route is the direction I have headed for allot of the reasons 
that you described. The only wrinkle is we are still on Kafka 0.7.3 so I am 
in the process of back porting this patch: 
https://github.com/linkedin/camus/commit/87917a2aea46da9d21c8f67129f6463af52f7aa8
 that 
is described 
here: https://groups.google.com/forum/#!topic/camus_etl/VcETxkYhzg8 -- so 
that we can handle reading and writing non-avro'ized (if that is a word) 
data.

I hope to have that done sometime in the morning and would be happy to 
share it if others can benefit from it.

Thanks,
Andrew


On Thursday, August 8, 2013 7:18:27 PM UTC-6, Felix GV wrote:

 The contrib code is simple and probably wouldn't require too much work to 
 fix, but it's a lot less robust than Camus, so you would ideally need to do 
 some work to make it solid against all edge cases, failure scenarios and 
 performance bottlenecks...

 I would definitely recommend investing in Camus instead, since it already 
 covers a lot of the challenges I'm mentioning above, and also has more 
 community support behind it at the moment (as far as I can tell, anyway), 
 so it is more likely to keep getting improvements than the contrib code.

 --
 Felix


 On Thu, Aug 8, 2013 at 9:28 AM, psaltis...@gmail.com javascript:wrote:

 We also have a need today to ETL from Kafka into Hadoop and we do not 
 currently nor have any plans to use Avro.

 So is the official direction based on this discussion to ditch the Kafka 
 contrib code and direct people to use Camus without Avro as Ken described 
 or are both solutions going to survive?

 I can put time into the contrib code and/or work on documenting the 
 tutorial on how to make Camus work without Avro.

 Which is the preferred route, for the long term?

 Thanks,
 Andrew

 On Wednesday, August 7, 2013 10:50:53 PM UTC-6, Ken Goodhope wrote:
  Hi Andrew,
 
 
 
  Camus can be made to work without avro. You will need to implement a 
 message decoder and and a data writer.   We need to add a better tutorial 
 on how to do this, but it isn't that difficult. If you decide to go down 
 this path, you can always ask questions on this list. I try to make sure 
 each email gets answered. But it can take me a day or two.
 
 
 
  -Ken
 
 
 
  On Aug 7, 2013, at 9:33 AM, ao...@wikimedia.org javascript: wrote:
 
 
 
   Hi all,
 
  
 
   Over at the Wikimedia Foundation, we're trying to figure out the best 
 way to do our ETL from Kafka into Hadoop.  We don't currently use Avro and 
 I'm not sure if we are going to.  I came across this post.
 
  
 
   If the plan is to remove the hadoop-consumer from Kafka contrib, do 
 you think we should not consider it as one of our viable options?
 
  
 
   Thanks!
 
   -Andrew
 
  
 
   --
 
   You received this message because you are subscribed to the Google 
 Groups Camus - Kafka ETL for Hadoop group.
 
   To unsubscribe from this group and stop receiving emails from it, 
 send an email to camus_etl+...@googlegroups.com javascript:.
 
   For more options, visit https://groups.google.com/groups/opt_out.