[jira] [Created] (KAFKA-4678) Create separate page for Connect docs

2017-01-20 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4678: -- Summary: Create separate page for Connect docs Key: KAFKA-4678 URL: https://issues.apache.org/jira/browse/KAFKA-4678 Project: Kafka Issue Type

Re: [VOTE] KIP-66: Single Message Transforms for Kafka Connect

2017-01-09 Thread Shikhar Bhushan
Thanks all. The vote passed with +5 (binding). On Fri, Jan 6, 2017 at 11:37 AM Shikhar Bhushan wrote: That makes sense to me, I'll fold that into the PR and update the KIP if it gets committed in that form. On Fri, Jan 6, 2017 at 9:44 AM Jason Gustafson wrote: +1 One minor comment: wou

Re: [VOTE] KIP-66: Single Message Transforms for Kafka Connect

2017-01-06 Thread Shikhar Bhushan
e the `init` method? > > On Thu, Jan 5, 2017 at 5:48 PM, Neha Narkhede wrote: > > > +1 (binding) > > > > On Wed, Jan 4, 2017 at 2:36 PM Shikhar Bhushan > > wrote: > > > > > I do plan on introducing a new `connect:transforms` module (which > > >

[jira] [Commented] (KAFKA-4598) Create new SourceTask commit callback method that takes offsets param

2017-01-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15805460#comment-15805460 ] Shikhar Bhushan commented on KAFKA-4598: Yeah, that's a reasonable al

Re: KafkaConnect SinkTask::put

2017-01-06 Thread Shikhar Bhushan
put it in the > wrong place? > > Thanks again, > David > > -----Original Message- > From: Shikhar Bhushan [mailto:shik...@confluent.io] > Sent: 05 January 2017 18:12 > To: dev@kafka.apache.org > Subject: Re: KafkaConnect SinkTask::put > > Hi David, > &

[jira] [Commented] (KAFKA-4598) Create new SourceTask commit callback method that takes offsets param

2017-01-05 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4598?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15802110#comment-15802110 ] Shikhar Bhushan commented on KAFKA-4598: In the meantime the workaround is to

[jira] [Created] (KAFKA-4598) Create new SourceTask commit callback method that takes offsets param

2017-01-05 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4598: -- Summary: Create new SourceTask commit callback method that takes offsets param Key: KAFKA-4598 URL: https://issues.apache.org/jira/browse/KAFKA-4598 Project

Re: KafkaConnect SinkTask::put

2017-01-05 Thread Shikhar Bhushan
Hi David, You can override the underlying consumer's `max.poll.records` setting for this. E.g. consumer.max.poll.records=500 Best, Shikhar On Thu, Jan 5, 2017 at 3:59 AM wrote: > Is there any way of limiting the number of events that are passed into the > call to the put(Collection) metho

[jira] [Commented] (KAFKA-4575) Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming source connector

2017-01-04 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4575?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15799807#comment-15799807 ] Shikhar Bhushan commented on KAFKA-4575: By the way, the error messag

[jira] [Updated] (KAFKA-4575) Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming sink connector

2017-01-04 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4575: --- Summary: Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming

[jira] [Work started] (KAFKA-4575) Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming source connector

2017-01-04 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4575 started by Shikhar Bhushan. -- > Transient failure in ConnectDistributedTest.test_pause_and_resume_sink

Re: [VOTE] KIP-66: Single Message Transforms for Kafka Connect

2017-01-04 Thread Shikhar Bhushan
On Wed, Jan 4, 2017 at 1:29 PM, Gwen Shapira wrote: > > > > > I would have preferred not to bundle transformations, but since SMT > > > capability is a much needed feature, I'll take it in its current form. > > > > > > +1 > > > > > >

[VOTE] KIP-66: Single Message Transforms for Kafka Connect

2017-01-04 Thread Shikhar Bhushan
Hi all, I'd like to start voting on KIP-66: https://cwiki.apache.org/confluence/display/KAFKA/KIP-66%3A+Single+Message+Transforms+for+Kafka+Connect Best, Shikhar

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2017-01-03 Thread Shikhar Bhushan
Makes sense Ewen, I edited the KIP to include this criteria. I'd like to start a voting thread soon unless anyone has additional points for discussion. On Fri, Dec 30, 2016 at 12:14 PM Ewen Cheslack-Postava wrote: On Thu, Dec 15, 2016 at 7:41 PM, Shikhar Bhushan wrote: > There is no

[jira] [Work started] (KAFKA-3209) Support single message transforms in Kafka Connect

2017-01-02 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3209 started by Shikhar Bhushan. -- > Support single message transforms in Kafka Conn

[jira] [Commented] (KAFKA-3513) Transient failure of OffsetValidationTest

2017-01-01 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3513?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15791790#comment-15791790 ] Shikhar Bhushan commented on KAFKA-3513: There was a failure in the last

[jira] [Assigned] (KAFKA-4575) Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming source connector

2016-12-29 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan reassigned KAFKA-4575: -- Assignee: Shikhar Bhushan > Transient failure

[jira] [Updated] (KAFKA-4575) Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming source connector

2016-12-29 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4575: --- Component/s: system tests KafkaConnect > Transient failure

[jira] [Created] (KAFKA-4575) Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming source connector

2016-12-29 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4575: -- Summary: Transient failure in ConnectDistributedTest.test_pause_and_resume_sink in consuming messages after resuming source connector Key: KAFKA-4575 URL: https

[jira] [Created] (KAFKA-4574) Transient failure in ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade with security_protocol = SASL_PLAINTEXT, SSL

2016-12-29 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4574: -- Summary: Transient failure in ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade with security_protocol = SASL_PLAINTEXT, SSL Key: KAFKA-4574 URL: https

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Shikhar Bhushan
ransformations. > > > > I really don't want to end up with 135 (or even 20) transformations in > > Kafka. So either we have a super-clear definition of what belongs and > what > > doesn't - or we put in one minimal example and the rest goes into the > > ecosystem. &g

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Shikhar Bhushan
ct later than to remove functionality. On Thu, Dec 15, 2016 at 11:59 AM, Shikhar Bhushan wrote: > I have updated KIP-66 > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 66%3A+Single+Message+Transforms+for+Kafka+Connect > with > the changes I proposed in the design. >

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-15 Thread Shikhar Bhushan
in Apache Kafka. What's the downside of keeping it out? Or at least keeping the built-in set super minimal (Flume has like 3 built-in interceptors)? Gwen On Wed, Dec 14, 2016 at 1:36 PM, Shikhar Bhushan wrote: > With regard to a), just using `ConnectRecord` with `newRecord` as a new >

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-14 Thread Shikhar Bhushan
e to do data enrichment by querying external systems, so building a bunch of transformations in could potentially open the floodgates, or at least make decisions about what is included vs what should be 3rd party muddy. -Ewen On Wed, Dec 7, 2016 at 11:46 AM, Shikhar Bhushan wrote: > Hi

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-12-12 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15743300#comment-15743300 ] Shikhar Bhushan commented on KAFKA-3209: Thanks [~snisarg]. I self-assigned i

[jira] [Created] (KAFKA-4524) ConfigDef.Type.LIST does not handle escaping

2016-12-12 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4524: -- Summary: ConfigDef.Type.LIST does not handle escaping Key: KAFKA-4524 URL: https://issues.apache.org/jira/browse/KAFKA-4524 Project: Kafka Issue Type

[jira] [Assigned] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-12-09 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan reassigned KAFKA-3209: -- Assignee: Shikhar Bhushan > Support single message transforms in Kafka Conn

[jira] [Commented] (KAFKA-3209) Support single message transforms in Kafka Connect

2016-12-07 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15729737#comment-15729737 ] Shikhar Bhushan commented on KAFKA-3209: [~snisarg] and [~jjchorrobe], I rev

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-12-07 Thread Shikhar Bhushan
ul 28, 2016 at 11:58 PM, Shikhar Bhushan wrote: > > > > > > Hmm, operating on ConnectRecords probably doesn't work since you need to > > emit the right type of record, which might mean instantiating a new one. > I > > think that means we either need 2 methods, o

[jira] [Updated] (KAFKA-3462) Allow SinkTasks to disable consumer offset commit

2016-11-14 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-3462: --- Resolution: Fixed Status: Resolved (was: Patch Available) This will be handled with

Re: [VOTE] KIP-89: Allow sink connectors to decouple flush and offset commit

2016-11-14 Thread Shikhar Bhushan
The vote passed with +3 binding votes. Thanks all! On Sun, Nov 13, 2016 at 1:42 PM Gwen Shapira wrote: +1 (binding) On Nov 9, 2016 2:17 PM, "Shikhar Bhushan" wrote: > Hi, > > I would like to initiate a vote on KIP-89 > > https://cwiki.apache.org/confluence/display

[jira] [Work started] (KAFKA-4161) Decouple flush and offset commits

2016-11-09 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4161 started by Shikhar Bhushan. -- > Decouple flush and offset comm

[VOTE] KIP-89: Allow sink connectors to decouple flush and offset commit

2016-11-09 Thread Shikhar Bhushan
Hi, I would like to initiate a vote on KIP-89 https://cwiki.apache.org/confluence/display/KAFKA/KIP-89%3A+Allow+sink+connectors+to+decouple+flush+and+offset+commit Best, Shikhar

[jira] [Assigned] (KAFKA-3910) Cyclic schema support in ConnectSchema and SchemaBuilder

2016-11-07 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3910?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan reassigned KAFKA-3910: -- Assignee: Shikhar Bhushan (was: Ewen Cheslack-Postava) > Cyclic schema support

[jira] [Commented] (KAFKA-4161) Decouple flush and offset commits

2016-11-04 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15637718#comment-15637718 ] Shikhar Bhushan commented on KAFKA-4161: Created KIP-89 for this h

[DISCUSS] KIP-89: Allow sink connectors to decouple flush and offset commit

2016-11-04 Thread Shikhar Bhushan
Hi all, I created KIP-89 for making a Connect API change that allows for sink connectors to decouple flush and offset commits. https://cwiki.apache.org/confluence/display/KAFKA/KIP-89%3A+Allow+sink+connectors+to+decouple+flush+and+offset+commit I'd welcome your input. Best, Shikhar

[jira] [Resolved] (KAFKA-4356) o.a.k.common.utils.SystemTime.sleep() swallows InterruptedException

2016-11-03 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4356?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-4356. Resolution: Duplicate > o.a.k.common.utils.SystemTime.sleep() swallows InterruptedExcept

[jira] [Commented] (KAFKA-4375) Kafka consumer may swallow some interrupts meant for the calling thread

2016-11-03 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4375?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15634044#comment-15634044 ] Shikhar Bhushan commented on KAFKA-4375: Good to have a report of this bei

[jira] [Created] (KAFKA-4356) o.a.k.common.utils.SystemTime.sleep() swallows InterruptedException

2016-10-28 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4356: -- Summary: o.a.k.common.utils.SystemTime.sleep() swallows InterruptedException Key: KAFKA-4356 URL: https://issues.apache.org/jira/browse/KAFKA-4356 Project: Kafka

[jira] [Resolved] (KAFKA-4342) Kafka-connect- support tinyint values

2016-10-25 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4342?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-4342. Resolution: Not A Problem The Connect schema type {{Schema.Type.INT8}} accurately maps to a

[jira] [Commented] (KAFKA-4306) Connect workers won't shut down if brokers are not available

2016-10-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4306?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15586568#comment-15586568 ] Shikhar Bhushan commented on KAFKA-4306: KAFKA-4154 is another issue relatin

[jira] [Updated] (KAFKA-3462) Allow SinkTasks to disable consumer offset commit

2016-10-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3462?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-3462: --- Issue Type: Improvement (was: Bug) > Allow SinkTasks to disable consumer offset com

[jira] [Updated] (KAFKA-4161) Decouple flush and offset commits

2016-10-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4161: --- Issue Type: Improvement (was: New Feature) > Decouple flush and offset comm

[jira] [Resolved] (KAFKA-3906) Connect logical types do not support nulls.

2016-09-23 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3906?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-3906. Resolution: Fixed I think we should handle null values at the converter layer to avoid

[jira] [Reopened] (KAFKA-4183) Logical converters in JsonConverter don't properly handle null values

2016-09-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan reopened KAFKA-4183: Assignee: Shikhar Bhushan (was: Ewen Cheslack-Postava) [~rhauch] Reopening this, I

[jira] [Commented] (KAFKA-3906) Connect logical types do not support nulls.

2016-09-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3906?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15497533#comment-15497533 ] Shikhar Bhushan commented on KAFKA-3906: [~jcustenborder] did this come up in

[jira] [Updated] (KAFKA-4161) Decouple flush and offset commits

2016-09-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4161: --- Summary: Decouple flush and offset commits (was: Allow connectors to request flush via the

[jira] [Commented] (KAFKA-4161) Allow connectors to request flush via the context

2016-09-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15496820#comment-15496820 ] Shikhar Bhushan commented on KAFKA-4161: We could also implement KAFKA-3462

[jira] [Created] (KAFKA-4173) SchemaProjector should successfully project when source schema field is missing and target schema field is optional

2016-09-14 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4173: -- Summary: SchemaProjector should successfully project when source schema field is missing and target schema field is optional Key: KAFKA-4173 URL: https://issues.apache.org

[jira] [Commented] (KAFKA-4161) Allow connectors to request flush via the context

2016-09-14 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15491296#comment-15491296 ] Shikhar Bhushan commented on KAFKA-4161: bq. Probably worth clarifying whe

[jira] [Created] (KAFKA-4161) Allow connectors to request flush via the context

2016-09-13 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4161: -- Summary: Allow connectors to request flush via the context Key: KAFKA-4161 URL: https://issues.apache.org/jira/browse/KAFKA-4161 Project: Kafka Issue

[jira] [Created] (KAFKA-4159) Allow overriding producer & consumer properties at the connector level

2016-09-13 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4159: -- Summary: Allow overriding producer & consumer properties at the connector level Key: KAFKA-4159 URL: https://issues.apache.org/jira/browse/KAFKA-4159 Pro

[jira] [Updated] (KAFKA-4154) Kafka Connect fails to shutdown if it has not completed startup

2016-09-12 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4154: --- Fix Version/s: (was: 0.10.0.2) > Kafka Connect fails to shutdown if it has not comple

[jira] [Updated] (KAFKA-4154) Kafka Connect fails to shutdown if it has not completed startup

2016-09-12 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4154: --- Fix Version/s: (was: 0.10.1.0) 0.10.0.2 > Kafka Connect fails

[jira] [Created] (KAFKA-4154) Kafka Connect fails to shutdown if it has not completed startup

2016-09-12 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4154: -- Summary: Kafka Connect fails to shutdown if it has not completed startup Key: KAFKA-4154 URL: https://issues.apache.org/jira/browse/KAFKA-4154 Project: Kafka

[jira] [Resolved] (KAFKA-4048) Connect does not support RetriableException consistently for sinks

2016-09-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-4048. Resolution: Not A Problem Turns out all exceptions from {{task.flush()}} are treated as

[jira] [Commented] (KAFKA-3962) ConfigDef support for resource-specific configuration

2016-09-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3962?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15468352#comment-15468352 ] Shikhar Bhushan commented on KAFKA-3962: This is also realizable today by u

[jira] [Comment Edited] (KAFKA-3962) ConfigDef support for resource-specific configuration

2016-09-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3962?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15468352#comment-15468352 ] Shikhar Bhushan edited comment on KAFKA-3962 at 9/6/16 7:4

[jira] [Commented] (KAFKA-4127) Possible data loss

2016-09-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15468012#comment-15468012 ] Shikhar Bhushan commented on KAFKA-4127: Dupe of KAFKA-3968 > Possible da

[jira] [Resolved] (KAFKA-4127) Possible data loss

2016-09-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-4127. Resolution: Duplicate > Possible data loss > -- > >

[jira] [Closed] (KAFKA-4127) Possible data loss

2016-09-06 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan closed KAFKA-4127. -- > Possible data loss > -- > > Key: KAFKA-4127 >

[jira] [Created] (KAFKA-4115) Grow default heap settings for distributed Connect from 256M to 1G

2016-09-01 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4115: -- Summary: Grow default heap settings for distributed Connect from 256M to 1G Key: KAFKA-4115 URL: https://issues.apache.org/jira/browse/KAFKA-4115 Project: Kafka

[jira] [Work started] (KAFKA-4100) Connect Struct schemas built using SchemaBuilder with no fields cause NPE in Struct constructor

2016-08-29 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4100?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4100 started by Shikhar Bhushan. -- > Connect Struct schemas built using SchemaBuilder with no fields cause

[jira] [Created] (KAFKA-4100) Connect Struct schemas built using SchemaBuilder with no fields cause NPE in Struct constructor

2016-08-29 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4100: -- Summary: Connect Struct schemas built using SchemaBuilder with no fields cause NPE in Struct constructor Key: KAFKA-4100 URL: https://issues.apache.org/jira/browse/KAFKA-4100

[jira] [Resolved] (KAFKA-4068) FileSinkTask - use JsonConverter to serialize

2016-08-20 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4068?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-4068. Resolution: Not A Problem I was thinking JSON since it would be easy to serialize to a

[jira] [Updated] (KAFKA-4070) Implement a useful Struct.toString()

2016-08-20 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4070: --- Description: Logging of {{Struct}}'s does not currently provide any useful output, and

[jira] [Updated] (KAFKA-4070) Implement a useful Struct.toString()

2016-08-20 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4070: --- Description: Logging of {{Struct}}'s does not currently provide any useful output, and

[jira] [Created] (KAFKA-4070) Implement a useful Struct.toString()

2016-08-20 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4070: -- Summary: Implement a useful Struct.toString() Key: KAFKA-4070 URL: https://issues.apache.org/jira/browse/KAFKA-4070 Project: Kafka Issue Type

[jira] [Created] (KAFKA-4068) FileSinkTask - use JsonConverter to serialize

2016-08-19 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4068: -- Summary: FileSinkTask - use JsonConverter to serialize Key: KAFKA-4068 URL: https://issues.apache.org/jira/browse/KAFKA-4068 Project: Kafka Issue Type

[jira] [Updated] (KAFKA-4042) DistributedHerder thread can die because of connector & task lifecycle exceptions

2016-08-19 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4042: --- Fix Version/s: 0.10.1.0 > DistributedHerder thread can die because of connector &am

[jira] [Resolved] (KAFKA-3054) Connect Herder fail forever if sent a wrong connector config or task config

2016-08-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan resolved KAFKA-3054. Resolution: Done > Connect Herder fail forever if sent a wrong connector config or t

[jira] [Commented] (KAFKA-3054) Connect Herder fail forever if sent a wrong connector config or task config

2016-08-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3054?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15427362#comment-15427362 ] Shikhar Bhushan commented on KAFKA-3054: Addressing this in KAFKA-4042, w

[jira] [Updated] (KAFKA-4048) Connect does not support RetriableException consistently for sinks

2016-08-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4048: --- Description: We only allow for handling {{RetriableException}} from calls to {{SinkTask.put

[jira] [Updated] (KAFKA-4048) Connect does not support RetriableException consistently for sinks

2016-08-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4048?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4048: --- Summary: Connect does not support RetriableException consistently for sinks (was: Connect

[jira] [Updated] (KAFKA-4042) DistributedHerder thread can die because of connector & task lifecycle exceptions

2016-08-16 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4042: --- Component/s: KafkaConnect > DistributedHerder thread can die because of connector &am

[jira] [Created] (KAFKA-4048) Connect does not support RetriableException consistently for sources & sinks

2016-08-16 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4048: -- Summary: Connect does not support RetriableException consistently for sources & sinks Key: KAFKA-4048 URL: https://issues.apache.org/jira/browse/KAFKA-

[jira] [Updated] (KAFKA-4042) DistributedHerder thread can die because of connector & task lifecycle exceptions

2016-08-15 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4042: --- Description: As one example, there isn't exception handli

[jira] [Updated] (KAFKA-4042) DistributedHerder thread can die because of connector & task lifecycle exceptions

2016-08-15 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-4042: --- Summary: DistributedHerder thread can die because of connector & task lifecycle except

[jira] [Work started] (KAFKA-3054) Connect Herder fail forever if sent a wrong connector config or task config

2016-08-15 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3054 started by Shikhar Bhushan. -- > Connect Herder fail forever if sent a wrong connector config or t

[jira] [Assigned] (KAFKA-3054) Connect Herder fail forever if sent a wrong connector config or task config

2016-08-15 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan reassigned KAFKA-3054: -- Assignee: Shikhar Bhushan (was: jin xing) > Connect Herder fail forever if sen

Re: [VOTE] KIP-75 - Add per-connector Converters

2016-08-15 Thread Shikhar Bhushan
+1 (non-binding) On Mon, Aug 15, 2016 at 1:20 PM Ismael Juma wrote: > +1 (binding) > > On 15 Aug 2016 7:21 pm, "Ewen Cheslack-Postava" wrote: > > > I would like to initiate the voting process for KIP-75: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > > 75+-+Add+per-connector+Conve

[jira] [Work started] (KAFKA-4042) Missing error handling in Worker.startConnector() can cause Herder thread to die

2016-08-15 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4042?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4042 started by Shikhar Bhushan. -- > Missing error handling in Worker.startConnector() can cause Herder thr

[jira] [Created] (KAFKA-4042) Missing error handling in Worker.startConnector() can cause Herder thread to die

2016-08-15 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4042: -- Summary: Missing error handling in Worker.startConnector() can cause Herder thread to die Key: KAFKA-4042 URL: https://issues.apache.org/jira/browse/KAFKA-4042

[jira] [Created] (KAFKA-4010) ConfigDef.toRst() should create sections for each group

2016-08-01 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-4010: -- Summary: ConfigDef.toRst() should create sections for each group Key: KAFKA-4010 URL: https://issues.apache.org/jira/browse/KAFKA-4010 Project: Kafka

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-07-28 Thread Shikhar Bhushan
> > > Hmm, operating on ConnectRecords probably doesn't work since you need to > emit the right type of record, which might mean instantiating a new one. I > think that means we either need 2 methods, one for SourceRecord, one for > SinkRecord, or we'd need to limit what parts of the message you ca

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-07-28 Thread Shikhar Bhushan
me is just an example)? > > On Sat, Jul 23, 2016 at 11:13 PM, Ewen Cheslack-Postava > > wrote: > > > On Fri, Jul 22, 2016 at 12:58 AM, Shikhar Bhushan > > wrote: > > > > > flatMap() / supporting 1->n feels nice and general since filtering is > > j

Re: Changing hash algorithm to LogCleaner offset map

2016-07-24 Thread Shikhar Bhushan
we want > to deploy with FIPS 140-2 mode enabled using only RSA security providers. > With this settings it is not possible to use MD5. > > On Fri, Jul 22, 2016 at 8:49 PM, Shikhar Bhushan > wrote: > > > Not sure I understand the motivation to use a FIPS-compliant

Re: Changing hash algorithm to LogCleaner offset map

2016-07-22 Thread Shikhar Bhushan
Not sure I understand the motivation to use a FIPS-compliant hash function for log compaction -- what are the security ramifications? On Fri, Jul 22, 2016 at 2:56 PM Luciano Afranllie wrote: > A little bit of background first. > > We are trying to make a deployment of Kafka that is FIPS 140-2 (

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-07-22 Thread Shikhar Bhushan
flatMap() / supporting 1->n feels nice and general since filtering is just the case of going from 1->0 I'm not sure why we'd need to do any more granular offset tracking (like sub-offsets) for source connectors: after transformation of a given record to n records, all of those n should map to same

[jira] [Updated] (KAFKA-3962) ConfigDef support for resource-specific configuration

2016-07-13 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3962?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan updated KAFKA-3962: --- Description: It often comes up with connectors that you want some piece of configuration

[jira] [Created] (KAFKA-3962) ConfigDef support for resource-specific configuration

2016-07-13 Thread Shikhar Bhushan (JIRA)
Shikhar Bhushan created KAFKA-3962: -- Summary: ConfigDef support for resource-specific configuration Key: KAFKA-3962 URL: https://issues.apache.org/jira/browse/KAFKA-3962 Project: Kafka

Re: [VOTE] KIP-65 Expose timestamps to Connect

2016-06-28 Thread Shikhar Bhushan
g), provided that we make the usage of `Long/null` versus > > `long/-1` consistent. > > > > Ismael > > > > On Sat, Jun 25, 2016 at 12:42 AM, Gwen Shapira > wrote: > > > > > +1 > > > > > > On Fri, Jun 24, 2016 at 10:59 AM, Shikhar Bhushan

Re: [DISCUSS] KIP-65 Expose timestamps to Connect

2016-06-26 Thread Shikhar Bhushan
with what you decide. > > Ismael > > On Sat, Jun 25, 2016 at 1:29 AM, Shikhar Bhushan > wrote: > > > Hi Ismael, > > > > Good point. This is down to an implementation detail, the getter was > added > > to the base class for `SourceRecord` and `SinkR

Re: [DISCUSS] KIP-65 Expose timestamps to Connect

2016-06-24 Thread Shikhar Bhushan
Ismael Juma wrote: > Hi Shikhar, > > Thanks for the KIP. One question: > > SinkRecord takes a `long` timestamp, but then exposes it via a method that > returns `Long`. Is this correct? And if so, can you please explain the > reasoning? > > Ismael > > On Thu, J

[VOTE] KIP-65 Expose timestamps to Connect

2016-06-24 Thread Shikhar Bhushan
Since there isn't much to discuss with this KIP, bringing it to a vote KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-65%3A+Expose+timestamps+to+Connect Pull request: https://github.com/apache/kafka/pull/1537 Thanks, Shikhar

[jira] [Commented] (KAFKA-3846) Connect record types should include timestamps

2016-06-23 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3846?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15346891#comment-15346891 ] Shikhar Bhushan commented on KAFKA-3846: https://cwiki.apache.org/conflu

[DISCUSS] KIP-65 Expose timestamps to Connect

2016-06-23 Thread Shikhar Bhushan
Kafkarati, Here is a pretty straightforward proposal, for exposing timestamps that were added in Kafka 0.10 to the connect framework so connectors can make use of them: https://cwiki.apache.org/confluence/display/KAFKA/KIP-65%3A+Expose+timestamps+to+Connect Appreciate your thoughts! Shikhar

[jira] [Work started] (KAFKA-3846) Connect record types should include timestamps

2016-06-23 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3846?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3846 started by Shikhar Bhushan. -- > Connect record types should include timesta

[jira] [Assigned] (KAFKA-3846) Connect record types should include timestamps

2016-06-21 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3846?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Shikhar Bhushan reassigned KAFKA-3846: -- Assignee: Shikhar Bhushan (was: Ewen Cheslack-Postava) > Connect record types sho

[jira] [Commented] (KAFKA-3335) Kafka Connect hangs in shutdown hook

2016-05-18 Thread Shikhar Bhushan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15290333#comment-15290333 ] Shikhar Bhushan commented on KAFKA-3335: Currently {{start()}} looks

  1   2   >