[jira] [Created] (SPARK-3462) parquet pushdown for unionAll

2014-09-09 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-3462: - Summary: parquet pushdown for unionAll Key: SPARK-3462 URL: https://issues.apache.org/jira/browse/SPARK-3462 Project: Spark Issue Type: Improvement

[jira] [Commented] (SPARK-3462) parquet pushdown for unionAll

2014-09-10 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3462?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14128451#comment-14128451 ] Cody Koeninger commented on SPARK-3462: --- Created a PR for feedback.

[jira] [Commented] (SPARK-3462) parquet pushdown for unionAll

2014-09-10 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3462?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14128682#comment-14128682 ] Cody Koeninger commented on SPARK-3462: --- Tested this on a cluster against unions of

[jira] [Commented] (SPARK-3146) Improve the flexibility of Spark Streaming Kafka API to offer user the ability to process message before storing into BM

2014-11-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14195146#comment-14195146 ] Cody Koeninger commented on SPARK-3146: --- I think this PR is an elegant way to solve

[jira] [Commented] (SPARK-4196) Streaming + checkpointing yields NotSerializableException for Hadoop Configuration from saveAsNewAPIHadoopFiles ?

2014-11-04 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4196?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14196641#comment-14196641 ] Cody Koeninger commented on SPARK-4196: --- Have you tried replacing

[jira] [Created] (SPARK-4229) Create hadoop configuration in a consistent way

2014-11-04 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-4229: - Summary: Create hadoop configuration in a consistent way Key: SPARK-4229 URL: https://issues.apache.org/jira/browse/SPARK-4229 Project: Spark Issue Type:

[jira] [Updated] (SPARK-4229) Create hadoop configuration in a consistent way

2014-11-04 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4229?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Cody Koeninger updated SPARK-4229: -- Description: Some places use SparkHadoopUtil.get.conf, some create a new hadoop config.

[jira] [Commented] (SPARK-3146) Improve the flexibility of Spark Streaming Kafka API to offer user the ability to process message before storing into BM

2014-12-18 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14252146#comment-14252146 ] Cody Koeninger commented on SPARK-3146: --- From my point of view, the interceptor

[jira] [Commented] (SPARK-3146) Improve the flexibility of Spark Streaming Kafka API to offer user the ability to process message before storing into BM

2014-12-18 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14252210#comment-14252210 ] Cody Koeninger commented on SPARK-3146: --- (1) is important because MessageAndMetadata

[jira] [Commented] (SPARK-4122) Add library to write data back to Kafka

2014-12-18 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14252261#comment-14252261 ] Cody Koeninger commented on SPARK-4122: --- +1 for the idea of making this write from

[jira] [Commented] (SPARK-3146) Improve the flexibility of Spark Streaming Kafka API to offer user the ability to process message before storing into BM

2014-12-18 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14252299#comment-14252299 ] Cody Koeninger commented on SPARK-3146: --- Yes, for the specific case of kafka,

[jira] [Commented] (SPARK-3146) Improve the flexibility of Spark Streaming Kafka API to offer user the ability to process message before storing into BM

2014-12-18 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14252397#comment-14252397 ] Cody Koeninger commented on SPARK-3146: --- ((K, V), (topicAndPartition, offset)) is

[jira] [Commented] (SPARK-3146) Improve the flexibility of Spark Streaming Kafka API to offer user the ability to process message before storing into BM

2014-12-19 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-3146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14254100#comment-14254100 ] Cody Koeninger commented on SPARK-3146: --- This is a real problem for production use,

[jira] [Created] (SPARK-4964) Exactly-once semantics for Kafka

2014-12-24 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-4964: - Summary: Exactly-once semantics for Kafka Key: SPARK-4964 URL: https://issues.apache.org/jira/browse/SPARK-4964 Project: Spark Issue Type: Improvement

[jira] [Commented] (SPARK-4964) Exactly-once semantics for Kafka

2014-12-24 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14258670#comment-14258670 ] Cody Koeninger commented on SPARK-4964: --- Usage example of the dstream for the

[jira] [Commented] (SPARK-4960) Interceptor pattern in receivers

2014-12-29 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14260144#comment-14260144 ] Cody Koeninger commented on SPARK-4960: --- You're saying for an implementation that

[jira] [Commented] (SPARK-4964) Exactly-once semantics for Kafka

2015-01-24 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4964?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14290747#comment-14290747 ] Cody Koeninger commented on SPARK-4964: --- Design doc at

[jira] [Commented] (SPARK-2808) update kafka to version 0.8.2

2015-02-11 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14316937#comment-14316937 ] Cody Koeninger commented on SPARK-2808: --- I'm also kind of curious what the

[jira] [Commented] (SPARK-4960) Interceptor pattern in receivers

2015-01-05 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-4960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14265570#comment-14265570 ] Cody Koeninger commented on SPARK-4960: --- At that point, it sounds like you're

[jira] [Commented] (SPARK-6434) [Streaming][Kafka] CreateDirectStream for empty topics

2015-03-20 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14371330#comment-14371330 ] Cody Koeninger commented on SPARK-6434: --- Yeah, I didn't notice alberto had also

[jira] [Created] (SPARK-6434) [Streaming][Kafka] CreateDirectStream for empty topics

2015-03-20 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-6434: - Summary: [Streaming][Kafka] CreateDirectStream for empty topics Key: SPARK-6434 URL: https://issues.apache.org/jira/browse/SPARK-6434 Project: Spark Issue

[jira] [Commented] (SPARK-6249) Get Kafka offsets from consumer group in ZK when using direct stream

2015-03-10 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14355561#comment-14355561 ] Cody Koeninger commented on SPARK-6249: --- First, I don't think setting group.id

[jira] [Commented] (SPARK-6569) Kafka directInputStream logs what appear to be incorrect warnings

2015-03-27 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14384038#comment-14384038 ] Cody Koeninger commented on SPARK-6569: --- I set it as warn because an empty batch can

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-04-24 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14511092#comment-14511092 ] Cody Koeninger commented on SPARK-7122: --- Does this actually have anything to do with

[jira] [Commented] (SPARK-7255) spark.streaming.kafka.maxRetries not documented

2015-04-29 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14520403#comment-14520403 ] Cody Koeninger commented on SPARK-7255: --- I think that should be fine to document,

[jira] [Commented] (SPARK-6431) Couldn't find leader offsets exception when creating KafkaDirectStream

2015-04-06 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14481266#comment-14481266 ] Cody Koeninger commented on SPARK-6431: --- I think this got mis-diagnosed on the

[jira] [Commented] (SPARK-6569) Kafka directInputStream logs what appear to be incorrect warnings

2015-04-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14394457#comment-14394457 ] Cody Koeninger commented on SPARK-6569: --- Keep in mind that this message is in

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14571675#comment-14571675 ] Cody Koeninger commented on SPARK-7122: --- Try doing something very straightforward

[jira] [Created] (SPARK-8127) KafkaRDD optimize count() take() isEmpty()

2015-06-05 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-8127: - Summary: KafkaRDD optimize count() take() isEmpty() Key: SPARK-8127 URL: https://issues.apache.org/jira/browse/SPARK-8127 Project: Spark Issue Type:

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14572059#comment-14572059 ] Cody Koeninger commented on SPARK-7122: --- If you can try out this patch

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14571454#comment-14571454 ] Cody Koeninger commented on SPARK-7122: --- Nicolas, do you have an actual compilable

[jira] [Commented] (SPARK-8389) Expose KafkaRDDs offsetRange in Java and Python

2015-06-19 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14594196#comment-14594196 ] Cody Koeninger commented on SPARK-8389: --- Have you guys been looking at SPARK-8337 ,

[jira] [Commented] (SPARK-8389) Expose KafkaRDDs offsetRange in Java and Python

2015-06-16 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14588202#comment-14588202 ] Cody Koeninger commented on SPARK-8389: --- There's already a ticket for the Python

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-16 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14588211#comment-14588211 ] Cody Koeninger commented on SPARK-7122: --- It's certainly your prerogative to wait for

[jira] [Commented] (SPARK-8337) KafkaUtils.createDirectStream for python is lacking API/feature parity with the Scala/Java version

2015-06-16 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14588517#comment-14588517 ] Cody Koeninger commented on SPARK-8337: --- So one thing to keep in mind is that if the

[jira] [Commented] (SPARK-8389) Expose KafkaRDDs offsetRange in Java and Python

2015-06-16 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14588404#comment-14588404 ] Cody Koeninger commented on SPARK-8389: --- So on the java side, just so I'm clear, are

[jira] [Commented] (SPARK-8390) Update DirectKafkaWordCount examples to show how offset ranges can be used

2015-06-17 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14590720#comment-14590720 ] Cody Koeninger commented on SPARK-8390: --- Did we actually want to update the

[jira] [Commented] (SPARK-8389) Expose KafkaRDDs offsetRange in Java and Python

2015-06-17 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14591170#comment-14591170 ] Cody Koeninger commented on SPARK-8389: --- Static type doesn't really matter since

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-01 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14567917#comment-14567917 ] Cody Koeninger commented on SPARK-7122: --- So if you're setting up a 2 second batch

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-01 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14567865#comment-14567865 ] Cody Koeninger commented on SPARK-7122: --- When you say it's getting behind, how are

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-02 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14569125#comment-14569125 ] Cody Koeninger commented on SPARK-7122: --- That number of tasks sounds normal for that

[jira] [Commented] (SPARK-7122) KafkaUtils.createDirectStream - unreasonable processing time in absence of load

2015-06-02 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14569254#comment-14569254 ] Cody Koeninger commented on SPARK-7122: --- Am I reading this correctly that you're

[jira] [Commented] (SPARK-8833) Kafka Direct API support offset in zookeeper

2015-07-06 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-8833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14615056#comment-14615056 ] Cody Koeninger commented on SPARK-8833: --- this basic idea has been discussed before

[jira] [Commented] (SPARK-7827) kafka streaming NotLeaderForPartition Exception could not be handled normally

2015-05-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7827?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14556631#comment-14556631 ] Cody Koeninger commented on SPARK-7827: --- As TD said, it looks like it did retry 4

[jira] [Created] (SPARK-9475) Consistent hadoop config for external/*

2015-07-30 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-9475: - Summary: Consistent hadoop config for external/* Key: SPARK-9475 URL: https://issues.apache.org/jira/browse/SPARK-9475 Project: Spark Issue Type: Sub-task

[jira] [Created] (SPARK-9473) Consistent hadoop config for SQL

2015-07-30 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-9473: - Summary: Consistent hadoop config for SQL Key: SPARK-9473 URL: https://issues.apache.org/jira/browse/SPARK-9473 Project: Spark Issue Type: Sub-task

[jira] [Created] (SPARK-9472) Consistent hadoop config for streaming

2015-07-30 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-9472: - Summary: Consistent hadoop config for streaming Key: SPARK-9472 URL: https://issues.apache.org/jira/browse/SPARK-9472 Project: Spark Issue Type: Sub-task

[jira] [Created] (SPARK-9474) Consistent hadoop config for core

2015-07-30 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-9474: - Summary: Consistent hadoop config for core Key: SPARK-9474 URL: https://issues.apache.org/jira/browse/SPARK-9474 Project: Spark Issue Type: Sub-task

[jira] [Commented] (SPARK-9476) Kafka stream loses leader after 2h of operation

2015-07-31 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14649123#comment-14649123 ] Cody Koeninger commented on SPARK-9476: --- Those all look like info or warn level log

[jira] [Commented] (SPARK-9434) Need how-to for resuming direct Kafka streaming consumers where they had left off before getting terminated, OR actual support for that mode in the Streaming API

2015-07-29 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646043#comment-14646043 ] Cody Koeninger commented on SPARK-9434: --- Dmitry, the nabble link you posted

[jira] [Commented] (SPARK-9434) Need how-to for resuming direct Kafka streaming consumers where they had left off before getting terminated, OR actual support for that mode in the Streaming API

2015-07-29 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14646113#comment-14646113 ] Cody Koeninger commented on SPARK-9434: --- If you want to submit a PR with doc changes

[jira] [Commented] (SPARK-9780) In case of invalid initialization of KafkaDirectStream, NPE is thrown

2015-08-11 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9780?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14681886#comment-14681886 ] Cody Koeninger commented on SPARK-9780: --- Makes sense, traveling currently but I'll

[jira] [Commented] (SPARK-6249) Get Kafka offsets from consumer group in ZK when using direct stream

2015-08-13 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14695939#comment-14695939 ] Cody Koeninger commented on SPARK-6249: --- Regarding streaming stats, those should be

[jira] [Commented] (SPARK-9947) Separate Metadata and State Checkpoint Data

2015-08-14 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14697098#comment-14697098 ] Cody Koeninger commented on SPARK-9947: --- You already have access to offsets and can

[jira] [Commented] (SPARK-9947) Separate Metadata and State Checkpoint Data

2015-08-14 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14697166#comment-14697166 ] Cody Koeninger commented on SPARK-9947: --- You can't re-use checkpoint data across

[jira] [Commented] (SPARK-9947) Separate Metadata and State Checkpoint Data

2015-08-14 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14697193#comment-14697193 ] Cody Koeninger commented on SPARK-9947: --- Didn't you already say that you were saving

[jira] [Commented] (SPARK-6249) Get Kafka offsets from consumer group in ZK when using direct stream

2015-08-14 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14697188#comment-14697188 ] Cody Koeninger commented on SPARK-6249: --- If you want an api that has imprecise

[jira] [Commented] (SPARK-9476) Kafka stream loses leader after 2h of operation

2015-08-09 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14679380#comment-14679380 ] Cody Koeninger commented on SPARK-9476: --- The stacktraces you posted don't contain

[jira] [Commented] (SPARK-6249) Get Kafka offsets from consumer group in ZK when using direct stream

2015-08-13 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6249?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14695929#comment-14695929 ] Cody Koeninger commented on SPARK-6249: ---

[jira] [Commented] (SPARK-9059) Update Direct Kafka Word count examples to show the use of HasOffsetRanges

2015-07-18 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14632476#comment-14632476 ] Cody Koeninger commented on SPARK-9059: --- How is this different from SPARK-8390 ? I

[jira] [Commented] (SPARK-9215) Implement WAL-free Kinesis receiver that give at-least once guarantee

2015-07-21 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14635179#comment-14635179 ] Cody Koeninger commented on SPARK-9215: --- I left some comments in the doc. Is Chris

[jira] [Commented] (SPARK-11195) Exception thrown on executor throws ClassNotFound on driver

2015-10-23 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11195?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14971783#comment-14971783 ] Cody Koeninger commented on SPARK-11195: I'm seeing this on 1.5.1 as well > Exception thrown on

[jira] [Commented] (SPARK-11211) Kafka - offsetOutOfRange forces to largest

2015-10-23 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14971834#comment-14971834 ] Cody Koeninger commented on SPARK-11211: In an attempt to replicate, I used the following: -

[jira] [Commented] (SPARK-11693) spark kafka direct streaming exception

2015-11-13 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15004072#comment-15004072 ] Cody Koeninger commented on SPARK-11693: You've under-provisioned Kafka storage and / or Spark

[jira] [Commented] (SPARK-11045) Contributing Receiver based Low Level Kafka Consumer from Spark-Packages to Apache Spark Project

2015-10-10 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14951877#comment-14951877 ] Cody Koeninger commented on SPARK-11045: The comments regarding parallelism are not accurate.

[jira] [Commented] (SPARK-11045) Contributing Receiver based Low Level Kafka Consumer from Spark-Packages to Apache Spark Project

2015-10-11 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14952299#comment-14952299 ] Cody Koeninger commented on SPARK-11045: "in Receiver based model, the number of partitions is

[jira] [Commented] (SPARK-11045) Contributing Receiver based Low Level Kafka Consumer from Spark-Packages to Apache Spark Project

2015-10-11 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14952373#comment-14952373 ] Cody Koeninger commented on SPARK-11045: What you're saying doesn't address my point regarding

[jira] [Commented] (SPARK-11045) Contributing Receiver based Low Level Kafka Consumer from Spark-Packages to Apache Spark Project

2015-10-11 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14952444#comment-14952444 ] Cody Koeninger commented on SPARK-11045: Direct stream partitions are 1:1 with kafka

[jira] [Commented] (SPARK-5569) Checkpoints cannot reference classes defined outside of Spark's assembly

2015-10-06 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-5569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14945091#comment-14945091 ] Cody Koeninger commented on SPARK-5569: --- The gist I originally posted, linked at the top of this

[jira] [Created] (SPARK-10963) Make KafkaCluster api public

2015-10-06 Thread Cody Koeninger (JIRA)
Cody Koeninger created SPARK-10963: -- Summary: Make KafkaCluster api public Key: SPARK-10963 URL: https://issues.apache.org/jira/browse/SPARK-10963 Project: Spark Issue Type: Improvement

[jira] [Commented] (SPARK-11698) Add option to ignore kafka messages that are out of limit rate

2015-11-13 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-11698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15004050#comment-15004050 ] Cody Koeninger commented on SPARK-11698: That looks like a reasonable way to solve your

[jira] [Commented] (SPARK-10320) Kafka Support new topic subscriptions without requiring restart of the streaming context

2015-11-13 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15004052#comment-15004052 ] Cody Koeninger commented on SPARK-10320: This is for practical purposes blocked on SPARK-10963 >

[jira] [Commented] (SPARK-10320) Support new topic subscriptions without requiring restart of the streaming context

2015-08-27 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14717342#comment-14717342 ] Cody Koeninger commented on SPARK-10320: As I said on the list, the best way to

[jira] [Commented] (SPARK-10320) Support new topic subscriptions without requiring restart of the streaming context

2015-08-31 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14724074#comment-14724074 ] Cody Koeninger commented on SPARK-10320: " If you restart the job and specify a new offset, that

[jira] [Updated] (SPARK-10320) Kafka Support new topic subscriptions without requiring restart of the streaming context

2015-08-31 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Cody Koeninger updated SPARK-10320: --- Summary: Kafka Support new topic subscriptions without requiring restart of the streaming

[jira] [Commented] (SPARK-10320) Kafka Support new topic subscriptions without requiring restart of the streaming context

2015-08-31 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14724113#comment-14724113 ] Cody Koeninger commented on SPARK-10320: It's possible this might be solvable with a

[jira] [Commented] (SPARK-10320) Kafka Support new topic subscriptions without requiring restart of the streaming context

2015-09-01 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14726076#comment-14726076 ] Cody Koeninger commented on SPARK-10320: You would supply a function, similar to the way

[jira] [Commented] (SPARK-10320) Kafka Support new topic subscriptions without requiring restart of the streaming context

2015-09-01 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14725719#comment-14725719 ] Cody Koeninger commented on SPARK-10320: So if you're changing topics in an event handler, it's

[jira] [Commented] (SPARK-10320) Kafka Support new topic subscriptions without requiring restart of the streaming context

2015-09-16 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10320?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14790843#comment-14790843 ] Cody Koeninger commented on SPARK-10320: I don't think there's much benefit to multiple dstreams

[jira] [Commented] (SPARK-9472) Consistent hadoop config for streaming

2015-09-30 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14936866#comment-14936866 ] Cody Koeninger commented on SPARK-9472: --- You can get around this by passing in the hadoop context

[jira] [Commented] (SPARK-9472) Consistent hadoop config for streaming

2015-09-30 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14937483#comment-14937483 ] Cody Koeninger commented on SPARK-9472: --- I'd suggest patching your distribution then, I doubt

[jira] [Comment Edited] (SPARK-10732) Starting spark streaming from a specific point in time.

2015-09-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14903231#comment-14903231 ] Cody Koeninger edited comment on SPARK-10732 at 9/22/15 7:02 PM: - Yeah,

[jira] [Commented] (SPARK-10732) Starting spark streaming from a specific point in time.

2015-09-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14903231#comment-14903231 ] Cody Koeninger commented on SPARK-10732: Yeah, even if that gets implemented it will likely be at

[jira] [Commented] (SPARK-10734) DirectKafkaInputDStream uses the OffsetRequest.LatestTime to find the latest offset, however using the batch time would be more desireable.

2015-09-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10734?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14903106#comment-14903106 ] Cody Koeninger commented on SPARK-10734: as I explained in SPARK-10732 , kafka's getOffsetsBefore

[jira] [Commented] (SPARK-10732) Starting spark streaming from a specific point in time.

2015-09-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14903110#comment-14903110 ] Cody Koeninger commented on SPARK-10732: As I already said, kafka's implementation of

[jira] [Commented] (SPARK-10732) Starting spark streaming from a specific point in time.

2015-09-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-10732?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14902648#comment-14902648 ] Cody Koeninger commented on SPARK-10732: It doesn't make sense to decouple this from the

[jira] [Commented] (SPARK-5569) Checkpoints cannot reference classes defined outside of Spark's assembly

2015-09-22 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-5569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14902692#comment-14902692 ] Cody Koeninger commented on SPARK-5569: --- I'm still not clear on what you're doing in

[jira] [Commented] (SPARK-5569) Checkpoints cannot reference classes defined outside of Spark's assembly

2015-09-21 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-5569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14901106#comment-14901106 ] Cody Koeninger commented on SPARK-5569: --- The direct stream doesn't save offset ranges, it saves

[jira] [Commented] (SPARK-5569) Checkpoints cannot reference classes defined outside of Spark's assembly

2015-09-21 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-5569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14901364#comment-14901364 ] Cody Koeninger commented on SPARK-5569: --- You can typically use sbt's merge strategy to deal with

[jira] [Commented] (SPARK-5569) Checkpoints cannot reference classes defined outside of Spark's assembly

2015-09-21 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-5569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14901435#comment-14901435 ] Cody Koeninger commented on SPARK-5569: --- Yeah, spark-streaming can be marked as provided. Try to

[jira] [Commented] (SPARK-12177) Update KafkaDStreams to new Kafka 0.9 Consumer API

2015-12-08 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12177?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15046975#comment-15046975 ] Cody Koeninger commented on SPARK-12177: I really think this needs to be handled as a separate

[jira] [Commented] (SPARK-12103) KafkaUtils createStream with multiple topics -- does not work as expected

2015-12-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15037773#comment-15037773 ] Cody Koeninger commented on SPARK-12103: A cursory review of the Kafka project documentation

[jira] [Commented] (SPARK-12203) Add KafkaDirectInputDStream that directly pulls messages from Kafka Brokers using receivers

2015-12-09 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12203?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048879#comment-15048879 ] Cody Koeninger commented on SPARK-12203: Commented on the PR. I don't think this makes sense for

[jira] [Comment Edited] (SPARK-9059) Update Python Direct Kafka Word count examples to show the use of HasOffsetRanges

2015-12-09 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048904#comment-15048904 ] Cody Koeninger edited comment on SPARK-9059 at 12/9/15 4:19 PM:

[jira] [Commented] (SPARK-9059) Update Python Direct Kafka Word count examples to show the use of HasOffsetRanges

2015-12-09 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048938#comment-15048938 ] Cody Koeninger commented on SPARK-9059: --- To be clear, I think SPARK-8389 should make it possible to

[jira] [Commented] (SPARK-6051) Add an option for DirectKafkaInputDStream to commit the offsets into ZK

2015-12-09 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-6051?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048895#comment-15048895 ] Cody Koeninger commented on SPARK-6051: --- Responded on the mailing list, but for posterity's sake:

[jira] [Commented] (SPARK-9059) Update Python Direct Kafka Word count examples to show the use of HasOffsetRanges

2015-12-09 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-9059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15048904#comment-15048904 ] Cody Koeninger commented on SPARK-9059: --- HasOffsetRanges is explained here

[jira] [Commented] (SPARK-12103) KafkaUtils createStream with multiple topics -- does not work as expected

2015-12-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15038084#comment-15038084 ] Cody Koeninger commented on SPARK-12103: On the off chance you're acting in good faith, actually

[jira] [Reopened] (SPARK-12103) KafkaUtils createStream with multiple topics -- does not work as expected

2015-12-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Cody Koeninger reopened SPARK-12103: PR for doc change: https://github.com/apache/spark/pull/10132 > KafkaUtils createStream with

[jira] [Commented] (SPARK-12103) KafkaUtils createStream with multiple topics -- does not work as expected

2015-12-03 Thread Cody Koeninger (JIRA)
[ https://issues.apache.org/jira/browse/SPARK-12103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15037986#comment-15037986 ] Cody Koeninger commented on SPARK-12103: Knowing that kafka messages have a key and value isn't

  1   2   3   4   5   >