Vincent,
Can the KIP more explicitly say that this is opt-in, and that by default
nothing will change?
Randall
On Tue, Jan 16, 2018 at 11:18 PM, Ewen Cheslack-Postava
wrote:
> Vincent,
>
> I think with the addition of a configuration to control this for
> compatibility,
SimpleHeaderConverter makes a lot of sense to me. Agree that this
> > introduces overhead, but perhaps simply documenting an easy
> "optimization"
> > will be enough to help those who are concerned about it? Since the
> > connector-devs decide whether they'l
+1 (non-binding)
On Mon, Jan 8, 2018 at 7:09 PM, Gwen Shapira wrote:
> +1 binding
>
> On Mon, Jan 8, 2018 at 4:59 PM Ewen Cheslack-Postava
> wrote:
>
> > +1 binding. Thanks for the KIP!
> >
> > -Ewen
> >
> > On Mon, Jan 8, 2018 at 8:34 AM, Ted Yu
Randall Hauch created KAFKA-6433:
Summary: Connect distributed workers should fail if their config
is "incompatible" with leader's
Key: KAFKA-6433
URL: https://issues.apache.org/jira/browse/
place.
>
> -Ewen
>
> On Fri, Jan 5, 2018 at 8:10 AM, Randall Hauch <rha...@gmail.com> wrote:
>
> > Sönke, I'm happy with the current proposal.
> >
> > Ewen, the proposal allows any characters in the name as long as they are
> > properly escaped/encoded.
ised
> > address.
> > > So I decided to reuse the existing rest.advertised.host.name and
> > > rest.advertised.port options and add additional option
> > > rest.advertised.security.protocol to specify whether HTTP or HTTPS
> > should
> > &
[
https://issues.apache.org/jira/browse/KAFKA-6387?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-6387.
--
Resolution: Won't Fix
Because of the inability to maintain backward compatibility
ov 16, 2017 at 5:35 PM, Sönke Liebau <
> soenke.lie...@opencore.com>
> > wrote:
> >
> > > Sounds good. I've added a few sentences to this effect to the KIP.
> > >
> > > On Thu, Nov 16, 2017 at 5:02 PM, Randall Hauch <rha...@gmail.com>
>
to `ByteArrayConverter` so that by default the framework doesn't
have to do much effort if headers aren't used/needed.
Thoughts?
On Tue, Dec 26, 2017 at 11:47 AM, Randall Hauch <rha...@gmail.com> wrote:
> Does anyone have any thoughts about this proposal for Connect header
> support?
>
> On
Does anyone have any thoughts about this proposal for Connect header
support?
On Thu, Dec 21, 2017 at 4:14 PM, Randall Hauch <rha...@gmail.com> wrote:
> All,
>
> I've updated KIP-145 to reflect my proposal. The proposal addresses SMTs
> and a different HeaderConverter
All,
I've updated KIP-145 to reflect my proposal. The proposal addresses SMTs
and a different HeaderConverter default, but I'll be updating my PR (
https://github.com/apache/kafka/pull/4319) soon. Feedback is very welcome!
Best regards,
Randall
On Thu, Dec 14, 2017 at 10:20 AM, Randall Hauch
Randall Hauch created KAFKA-6387:
Summary: Worker's producer and consumer configs should inherit
from worker configs
Key: KAFKA-6387
URL: https://issues.apache.org/jira/browse/KAFKA-6387
Project
Thanks, Ewen. I think the KIP is clear enough about the intent and the
changed behavior.
On Tue, Dec 12, 2017 at 12:22 AM, Ewen Cheslack-Postava
wrote:
> And to clarify a bit further: the goal is for both standalone and
> distributed mode to display the same basic
after conversion is
> integer you can still ask for float and it will type concert if possible.
>
> Cheers
> Mike
>
>
> Sent from my iPhone
>
> > On 13 Dec 2017, at 01:36, Randall Hauch <rha...@gmail.com> wrote:
> >
> > Trying to revive this after sev
Trying to revive this after several months of inactivity
I've spent quite a bit of time evaluating the current KIP-145 proposal and
several of the suggested PRs. The original KIP-145 proposal is relatively
minimalist (which is very nice), and it adopts Kafka's approach to headers
where header
Randall Hauch created KAFKA-6313:
Summary: Kafka Core maven dependencies are missing SLF4J API
Key: KAFKA-6313
URL: https://issues.apache.org/jira/browse/KAFKA-6313
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-6279?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-6279.
--
Resolution: Duplicate
> Connect metrics do not get cleaned up for a source connec
Randall Hauch created KAFKA-6279:
Summary: Connect metrics do not get cleaned up for a source
connector that doesn't stop properly
Key: KAFKA-6279
URL: https://issues.apache.org/jira/browse/KAFKA-6279
gt; On Thu, Nov 16, 2017 at 2:59 PM, Sönke Liebau <soenke.lie...@opencore.com>
> wrote:
>
> > Hi Randall,
> >
> > I had mentioned this edge case in the KIP, but will add some further
> > detail to further clarify all changing scenarios post pull request.
> >
er.
>
> Let me know what you think.
>
> Kind regards,
> Sönke
>
> ᐧ
>
>> On Tue, Nov 14, 2017 at 7:03 PM, Randall Hauch <rha...@gmail.com> wrote:
>>
>> Thanks for updating the KIP to reflect the current process. However, I
>> still questi
Randall Hauch created KAFKA-6208:
Summary: Reduce startup time for Kafka Connect workers
Key: KAFKA-6208
URL: https://issues.apache.org/jira/browse/KAFKA-6208
Project: Kafka
Issue Type
uite small and probably won't
> even be noticed by many people..
>
> best regards,
> Sönke
>
> On Thu, Nov 9, 2017 at 9:10 PM, Randall Hauch <rha...@gmail.com> wrote:
>
> > Any progress on updating the PR and withdrawing KIP-212?
> >
> > On Fri, Oct 27,
Randall Hauch created KAFKA-6201:
Summary: Provide more context in Kafka Connect REST error messages
Key: KAFKA-6201
URL: https://issues.apache.org/jira/browse/KAFKA-6201
Project: Kafka
[
https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch reopened KAFKA-4827:
--
> Kafka connect: error with special characters in connector n
[
https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-4827.
--
Resolution: Won't Fix
To be clear, connector names should be *URL-encoded* in the REST request
[
https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch reopened KAFKA-4827:
--
> Kafka connect: error with special characters in connector n
[
https://issues.apache.org/jira/browse/KAFKA-4938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-4938.
--
Resolution: Duplicate
I'm closing this issue, since KAFKA-4930 is actually closer to the heart
[
https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-4827.
--
Resolution: Fixed
I'm closing this issue, since KAFKA-4930 is actually closer to the heart
[
https://issues.apache.org/jira/browse/KAFKA-5669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5669.
--
Resolution: Duplicate
> Define allowed characters for connector na
[
https://issues.apache.org/jira/browse/KAFKA-5669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5669.
--
Resolution: Won't Fix
I'm closing this issue, since KAFKA-4930 is actually closer to the heart
[
https://issues.apache.org/jira/browse/KAFKA-5669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch reopened KAFKA-5669:
--
> Define allowed characters for connector na
Any progress on updating the PR and withdrawing KIP-212?
On Fri, Oct 27, 2017 at 5:19 PM, Randall Hauch <rha...@gmail.com> wrote:
> Yes, connector names should not be blank or contain just whitespace. In
> fact, I might recommend that we trim whitespace at the front and rear of
>
;
> > > > > Thanks all for looking this over.
> > > > >
> > > > > On Fri, Nov 3, 2017 at 7:31 PM, Jason Gustafson <
> ja...@confluent.io>
> > > > > wrote:
> > > > >
> > > > &g
Randall Hauch created KAFKA-6168:
Summary: Connect Schema comparison is slow for large schemas
Key: KAFKA-6168
URL: https://issues.apache.org/jira/browse/KAFKA-6168
Project: Kafka
Issue Type
+1 (non-binding)
Thanks for pushing this through. Great work!
Randall Hauch
On Wed, Nov 1, 2017 at 9:40 AM, Jeff Klukas <jklu...@simple.com> wrote:
> I haven't heard any additional concerns over the proposal, so I'd like to
> get the voting process started for:
>
> https:
Randall Hauch created KAFKA-6142:
Summary: Connect worker configurations and connector
configurations should accept environment variables
Key: KAFKA-6142
URL: https://issues.apache.org/jira/browse/KAFKA-6142
> I will further investigate which characters the url decoding that
> jetty
> > > > brings to the table will let us use and if all of these are correctly
> > > > handled during connector creation and report back with a new list of
> > > > characters tha
[
https://issues.apache.org/jira/browse/KAFKA-6133?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-6133.
--
Resolution: Invalid
Closing this as INVALID, since this is not Apache Kafka code
Here's the link to KIP-212:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=74684586
I do think it's worthwhile to define the rules for connector names.
However, I think it would be better to describe the current restrictions
for names outside of them appearing within URLs. For
Very interesting. Would the proposed configuration provider be set at the
connector level or the worker level? The latter would obviously be required
to handle all/multiple connector configurations. Either way, the provider
class(es) would need to be installed on the worker (really, every worker),
Sounds like you're using Kafka Connect and trying to write a source
connector. The Connect framework separates responsibilities:
1. Your source connector is responsible for generating the SourceRecord
objects and returning them to the Connect framework via your task's
"poll()" method.
2. The
ameter with
> comma
> > > separated list. What do you think?
> > >
> > > As for the "rest" prefix - if we remove it, some of the same
> > configuration
> > > options are already used today as the option for connecting from Kafka
> > > Conne
to support both "http" and "https"
simultaneously, so IMO we should add that from the beginning.
On Fri, Oct 13, 2017 at 9:34 AM, Randall Hauch <rha...@gmail.com> wrote:
> It'd be useful to specify the default values for the configuration
> properties.
>
> On Tue,
[
https://issues.apache.org/jira/browse/KAFKA-4575?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-4575.
--
Resolution: Fixed
This appears to no longer be an issue, so I'm closing this as fixed
It'd be useful to specify the default values for the configuration
properties.
On Tue, Oct 10, 2017 at 2:53 AM, Jakub Scholz wrote:
> FYI: Based on Ewen's suggestion from the related JIRA, I added a
> clarification to the KIP that it doesn't do anything around authorization /
>
On Thu, Oct 12, 2017 at 1:59 AM, Stephane Maarek <
steph...@simplemachines.com.au> wrote:
> Hi,
>
>
>
> I had a look at the Connect Source Worker code and have two questions:
> When a Source Task commits offsets, does it perform compaction /
> optimisation before sending off? E.g. I read from 1
[
https://issues.apache.org/jira/browse/KAFKA-5774?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5774.
--
Resolution: Fixed
Fix Version/s: 1.0.0
Indeed, this was [fixed with this PR|https
[
https://issues.apache.org/jira/browse/KAFKA-2376?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-2376.
--
Resolution: Fixed
All of the subtasks have been completed, so marking this as fixed.
>
[
https://issues.apache.org/jira/browse/KAFKA-5904?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5904.
--
Resolution: Fixed
Resolved as part of the [PR](https://github.com/apache/kafka/pull/4011
eers
>
>> On Wed, Oct 4, 2017 at 9:58 AM, Randall Hauch <rha...@gmail.com> wrote:
>>
>> Currently the KIP's scope is only topics that don't yet exist, and we have
>> to cognizant of race conditions between tasks with the same connector. I
>> think it is wo
pose
> of ensuring auto-created topics have the right config.
>
> Mathieu
>
>
> On Tue, May 23, 2017 at 11:38 AM, Randall Hauch <rha...@gmail.com>
> wrote:
> > Thanks for the quick feedback, Mathieu. Yes, the first configuration
> rule
>
Randall Hauch created KAFKA-5990:
Summary: Add generated documentation for Connect metrics
Key: KAFKA-5990
URL: https://issues.apache.org/jira/browse/KAFKA-5990
Project: Kafka
Issue Type
Randall Hauch created KAFKA-5987:
Summary: Kafka metrics templates used in document generation
should maintain order of tags
Key: KAFKA-5987
URL: https://issues.apache.org/jira/browse/KAFKA-5987
[
https://issues.apache.org/jira/browse/KAFKA-5954?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5954.
--
Resolution: Fixed
> Failure in Connect system test: ConnectRestApiT
Randall Hauch created KAFKA-5954:
Summary: Failure in Connect system test: ConnectRestApiTest
Key: KAFKA-5954
URL: https://issues.apache.org/jira/browse/KAFKA-5954
Project: Kafka
Issue Type
Randall Hauch created KAFKA-5900:
Summary: Create Connect metrics common to source and sink tasks
Key: KAFKA-5900
URL: https://issues.apache.org/jira/browse/KAFKA-5900
Project: Kafka
Issue
Randall Hauch created KAFKA-5902:
Summary: Create Connect metrics for sink tasks
Key: KAFKA-5902
URL: https://issues.apache.org/jira/browse/KAFKA-5902
Project: Kafka
Issue Type: Sub-task
Randall Hauch created KAFKA-5901:
Summary: Create Connect metrics for source tasks
Key: KAFKA-5901
URL: https://issues.apache.org/jira/browse/KAFKA-5901
Project: Kafka
Issue Type: Sub-task
Randall Hauch created KAFKA-5903:
Summary: Create Connect metrics for workers
Key: KAFKA-5903
URL: https://issues.apache.org/jira/browse/KAFKA-5903
Project: Kafka
Issue Type: Sub-task
Randall Hauch created KAFKA-5899:
Summary: Create Connect metrics for connectors
Key: KAFKA-5899
URL: https://issues.apache.org/jira/browse/KAFKA-5899
Project: Kafka
Issue Type: Sub-task
187+-+Add+cumulative+count+metric+for+all+Kafka+rate+metrics
On Wed, Sep 13, 2017 at 9:53 PM, Randall Hauch <rha...@gmail.com> wrote:
> The KIP has passed with three binding +1 votes (Gwen, Sriram, Jason) and
> no -1 or +0 votes.
>
> Thanks to everyone for the feedback.
>
Subramanian <r...@confluent.io>
> wrote:
>
> > +1
> >
> > On Tue, Sep 12, 2017 at 12:41 PM, Gwen Shapira <g...@confluent.io>
> wrote:
> >
> > > My +1 remains :)
> > >
> > > On Tue, Sep 12, 2017 at 12:31 PM Randall Hauch <rha...@
t;
> > > > Gwen
> > > >
> > > > On Mon, Sep 11, 2017 at 7:50 PM Ewen Cheslack-Postava <
> > e...@confluent.io
> > > >
> > > > wrote:
> > > >
> > > > > re: questions about additional metrics, I think we
o> wrote:
>
> > +1
> >
> > Thanks for this. Can't wait for more complete monitoring for Connect.
> >
> > On Mon, Sep 11, 2017 at 7:40 AM Randall Hauch <rha...@gmail.com> wrote:
> >
> > > I'd like to start the vote on KIP-196 to add metrics to
Okay, I think I've incorporated all feedback except for Gwen and Roger than
would like to have timing metrics. Given the deadline and Ewen's concern
about degraded performance, I think it's prudent to leave those out of this
KIP and proceed as is.
On Tue, Sep 12, 2017 at 12:48 PM, Randall Hauch
t; > implementation just does something inefficient), but I'm not really sure
> > about that yet.
> >
> > -Ewen
> >
> > On Mon, Sep 11, 2017 at 7:11 PM, Randall Hauch <rha...@gmail.com> wrote:
> >
> > > Based on Roger and Ewen's feedback, I
e attributes
> are named identically ("sink-record-lag-avg" instead of
> "sink-record-{topic}-{partition}.records-lag-avg"), so monitoring systems
> have a consistent string they can use, instead of needing to
> prefix-and-suffix matching against the attribute name. And T
his, including offset
> delete, is to stop the connector, apply operation, then continue.
>
ack.
>
> -Ewen
>
>
> On Sun, Sep 10, 2017 at 3:12 PM, Ted Yu <yuzhih...@gmail.com> wrote:
>
> > bq. How about calling it kafka-connect-source-offset-tool.sh
> >
> > This is bett
On Mon, Sep 11, 2017 at 2:56 PM, Gwen Shapira <g...@confluent.io> wrote:
>
> > +1
> >
> > On Mon, Sep 11, 2017 at 1:33 PM Ted Yu <yuzhih...@gmail.com> wrote:
> >
> > > +1
> > >
> > > On Mon, Sep 11, 2017 at 7:43 AM, Randall Hauc
count/total metrics are only valid since the most
recent rebalance and are therefore less meaningful, and were removed.
On Mon, Sep 11, 2017 at 6:50 PM, Randall Hauch <rha...@gmail.com> wrote:
> Thanks, Ewen. Comments inline below.
>
> On Mon, Sep 11, 2017 at 5:46 PM, Ewen Ches
Ack.
> * Why worker-level offset commit stats? It's not clear to me that these are
> useful without considering the specific connector.
>
So would they make more sense on the tasks? Again, on the worker they're
aggregates.
>
> -Ewen
>
>
> On Mon, Sep 11, 2017 at 9:43 AM, Randall
stinction in the Connect framework.
> - Batching stats
> - Histogram of flush batch size
> - Counts of flush trigger method (time vs max batch size)
>
Should be able to add these.
>
> Cheers,
>
> Roger
>
> On Sun, Sep 10, 2017 at 8:
I'd like to start the vote on KIP-199 to add a command line tool that will
allow Connect operators to read, modify, and update source connector
offsets. Details are here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-199%3A+Add+Kafka+Connect+offset+tool
Thanks, and best regards.
Randall
I'd like to start the vote on KIP-196 to add metrics to the Kafka Connect
framework so the worker processes can be measured. Details are here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework
Thanks, and best regards.
Randall
Randall Hauch created KAFKA-5867:
Summary: Kafka Connect applications should log info message when
starting up
Key: KAFKA-5867
URL: https://issues.apache.org/jira/browse/KAFKA-5867
Project: Kafka
> Better note this in the KIP.
>
Addressed in a few places. Hopefully it is more clear now.
>
> bq. but will remove the offsets for the partition with file "b"
>
> Please move the sample JSON below the above description.
>
Done.
>
> Cheers
>
> On Sun, Sep 10
Hi all,
KIP-199 [1] describes a new tool that will allow Connect operators to read,
update, and remove offsets stored by Connect runtime. This capability has
been often asked for by Connect users. The proposal is simple but flexible.
Please review and add feedback.
Best regards,
Randall
[1]
ortant to know both numbers.
>
>
> Gwen
>
> On Thu, Sep 7, 2017 at 7:50 PM, Randall Hauch <rha...@gmail.com> wrote:
>
> > Hi everyone.
> >
> > I've created a new KIP to add metrics to the Kafka Connect framework:
> > https://cwiki.apache.org/conf
[
https://issues.apache.org/jira/browse/KAFKA-5798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5798.
--
Resolution: Invalid
Closing as invalid, since this involves connectors that are not part
AM Florian Hussonnois <fhussonn...@gmail.com>
> wrote:
>
> > Hi Shapira,
> >
> > We only expose the OffsetStorageReader to connector which relies on
> > KafkaOffsetBackingStore. The store continuesly consumes offsets from
> kafka
> > so I think we can't have
+1 (non-binding)
Randall
On Fri, Sep 8, 2017 at 6:32 PM, Ted Yu wrote:
> Hi,
> Please take a look at the following and cast your vote:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 197+Connect+REST+API+should+include+the+connector+type+
>
Looks good to me! Thanks again!
I say go ahead and ask for a vote in a new thread.
Randall
On Fri, Sep 8, 2017 at 6:22 PM, Ted Yu <yuzhih...@gmail.com> wrote:
> Lowercase constants are generated by the enum.
>
> Updated KIP again.
>
> On Fri, Sep 8, 2017 at 4:08 P
the KIP accordingly.
>
> Cheers
>
> On Fri, Sep 8, 2017 at 3:57 PM, Randall Hauch <rha...@gmail.com> wrote:
>
> > Hi, Ted. Thanks for the quick turn around.
> >
> > I didn't remember that the response to the "/connectors/{name}/config&q
; wrote:
> Thanks for the reminder, Randall.
>
> I have modified the KIP to include these two endpoints.
>
> On Fri, Sep 8, 2017 at 11:00 AM, Randall Hauch <rha...@gmail.com> wrote:
>
> > Hi, Ted.
> >
> > Thanks for creating this KIP and for
Hi, Ted.
Thanks for creating this KIP and for working on the implementation. The
proposal looks great for the "/connectors/{name}" endpoint, but there are
several others that we need to consider as well so that the responses are
all consistent. In particular, look at "/connectors/{name}/status"
>
>> On Thu, Sep 7, 2017 at 5:04 PM Gwen Shapira <g...@confluent.io> wrote:
>>
>> +1 (binding)
>>
>> Looking forward to see how connector implementations use this in practice
>> :)
>>
>>> On Thu, Sep 7, 2017 at 3:49 PM Randall Hauch <
Hi everyone.
I've created a new KIP to add metrics to the Kafka Connect framework:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework
The KIP approval deadline is looming, so if you're interested in Kafka
Connect metrics please review and provide
I'd like to open the vote for KIP-131:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-131+-+Add+access+to+OffsetStorageReader+from+SourceConnector
Thanks to Florian for submitting the KIP and the implementation, and to
everyone else that helped review.
Best regards,
Randall
I will update the KIP and the PR with your last approach which sounds
> better.
>
> Thanks.
>
> Le 16 août 2017 00:53, "Randall Hauch" <rha...@gmail.com> a écrit :
>
> Sorry it's taken me so long to come back to this.
>
> Have you considered creating a `So
Randall Hauch created KAFKA-5774:
Summary: WorkerSinkTask failure exception should include cause
Key: KAFKA-5774
URL: https://issues.apache.org/jira/browse/KAFKA-5774
Project: Kafka
Issue
Randall Hauch created KAFKA-5741:
Summary: Prioritize threads in Connect distributed worker process
Key: KAFKA-5741
URL: https://issues.apache.org/jira/browse/KAFKA-5741
Project: Kafka
Issue
Randall Hauch created KAFKA-5740:
Summary: Use separate file for HTTP logs
Key: KAFKA-5740
URL: https://issues.apache.org/jira/browse/KAFKA-5740
Project: Kafka
Issue Type: Improvement
Sorry it's taken me so long to come back to this.
Have you considered creating a `SourceConnectorContext` interface that
extends `ConnectorContext` and that adds the method to access the offset
storage? This would very closely match the existing `SourceTaskContext`.
`SourceConnector`
Randall Hauch created KAFKA-5657:
Summary: Connect REST API should include the connector type when
describing a connector
Key: KAFKA-5657
URL: https://issues.apache.org/jira/browse/KAFKA-5657
Project
[
https://issues.apache.org/jira/browse/KAFKA-5539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch resolved KAFKA-5539.
--
Resolution: Invalid
Kafka Connect's plugin path feature was added in 0.11.0.0, and does
Randall Hauch created KAFKA-5472:
Summary: Connector validate REST endpoint returning duplicate
entries in "groups"
Key: KAFKA-5472
URL: https://issues.apache.org/jira/browse/KAFKA-5472
[
https://issues.apache.org/jira/browse/KAFKA-5451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16049712#comment-16049712
]
Randall Hauch commented on KAFKA-5451:
--
KAFKA-5450 demonstrates a case where shorter Kafka Connect
Randall Hauch created KAFKA-5451:
Summary: Kafka Connect should scan classpath asynchronously
Key: KAFKA-5451
URL: https://issues.apache.org/jira/browse/KAFKA-5451
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-5450?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16049699#comment-16049699
]
Randall Hauch edited comment on KAFKA-5450 at 6/14/17 9:46 PM:
---
Added
[
https://issues.apache.org/jira/browse/KAFKA-5450?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Randall Hauch updated KAFKA-5450:
-
Status: Patch Available (was: Open)
Added the patch that changes the timeout in the Connect
501 - 600 of 700 matches
Mail list logo