ctive here not to change the existing protocol and still be able
> to write and read logs from remote storage.
>
>
> -Harsha
>
> On Feb 4, 2019, 2:53 PM -0800, Ryanne Dolan ,
> wrote:
> > Thanks Harsha, makes sense for the most part.
> >
> > > tiered storag
-- the broker
> wouldn't necessarily care if it reads from file:// or s3:// to load a given
> segment."
>
> Yes, this is what we are discussing in KIP. We are leaving the details of
> loading segments to RLM read part instead of directly exposing this in the
> Broker. This
Harsha, Sriharsha, Suresh, a couple thoughts:
- How could this be used to leverage fast key-value stores, e.g. Couchbase,
which can serve individual records but maybe not entire segments? Or is the
idea to only support writing and fetching entire segments? Would it make
sense to support both?
-
would have thought that the connector
> implementor would want to provide only a single variant of commitRecord().
>
> Andrew Schofield
> IBM Event Streams
>
> On 31/01/2019, 03:00, "Ryanne Dolan" wrote:
>
> I've updated the KIP and PR to overload commitR
I've updated the KIP and PR to overload commitRecord instead of adding a
new method. Here's the PR:
https://github.com/apache/kafka/pull/6171
Ryanne
On Mon, Jan 21, 2019 at 6:29 PM Ryanne Dolan wrote:
> Andrew Schofield suggested we overload the commitRecord method instead of
> adding
Pippin, thanks for your interest. I will publish a PR soon (several
days?) which you'll be able to build and play with. Watch this space
:)
Ryanne
On Thu, Jan 24, 2019 at 5:19 PM Pippin Wallace wrote:
>
> I see that the Current state of KIP-382 recently changed from Voting to
> Accepted on
is a signal to stop polling. It's basically a
>> request from the framework to the task and it doesn't tell the task that
>> it's actually finished. One of the purposes of the KC framework is to make
>> life easy for a connector developer and a nice clean "all done now"
s actually finished. One of the purposes of the KC framework is to make
> life easy for a connector developer and a nice clean "all done now" method
> would help.
>
> I think I'll add a diagram to illustrate to the KIP.
>
> Andrew Schofield
> IBM Event Stre
+1 non-binding, thanks!
Ryanne
On Tue, Jan 22, 2019 at 11:38 AM te...@confluent.io wrote:
>
> Hi all,
>
> We would like to start vote on KIP-421 to to enhance the AbstractConfig base
> class to support replacing variables in configurations just prior to parsing
> and validation.
>
> Link for
> Thanks for the KIP and patient discussion. +1 from me as well.
>
> Jiangjie (Becket) Qin
>
> On Fri, Jan 11, 2019 at 1:11 AM Jun Rao wrote:
>>
>> Hi, Ryanne,
>>
>> Thanks for the explanation. All make sense to me now. +1 on the KIP from me.
>>
&g
l is something we can all agree to. Let's start simple and learn
> from our users whether or not they need more flexibility and control.
>
> Please respond with your thoughts. Thanks!
>
> Best regards,
>
> Randall
>
> On Tue, Nov 27, 2018 at 7:36 PM Ryanne Dolan
> wrote:
&
Andrew Schofield suggested we overload the commitRecord method instead of
adding a new one. Thoughts?
Ryanne
On Thu, Jan 17, 2019, 5:34 PM Ryanne Dolan I had to change the KIP number (concurrency is hard!) so the link is now:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/K
tRecord() in my view.
>
> Just my 2 cents.
>
> Andrew Schofield
> IBM Event Streams
>
>
> On 17/01/2019, 23:54, "Ryanne Dolan" wrote:
>
> Hey y'all, please vote for KIP-416 by replying +1 to this thread.
>
> Right now, there is no way f
+1 (non-binding)
But I suggest:
- drop "get" from getOffset, getTimestamp.
- add to the motivation section why this is better than constructing a
KafkaConsumer and using seek(), commit() etc.
- add some rejected alternatives.
Ryanne
On Mon, Jan 21, 2019, 7:57 AM Dongjin Lee We have +4
Andrew, do we know whether the SourceTask may be start()ed again? If this
is the last call to a SourceTask I suggest we call it close(). I can't tell
from the documentation.
Also, do we need this if a SourceTask can keep track of whether it was
start()ed since the last stop()?
Ryanne
On Fri,
Hey y'all, please vote for KIP-416 by replying +1 to this thread.
Right now, there is no way for a SourceConnector/Task to know:
- whether a record was successfully sent to Kafka, vs filtered out or
skipped.
- the downstream offsets and metadata of sent records
KIP-416 proposes adding a
I had to change the KIP number (concurrency is hard!) so the link is now:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-416%3A+Notify+SourceTask+of+ACK%27d+offsets%2C+metadata
Ryanne
On Fri, Jan 11, 2019 at 2:43 PM Ryanne Dolan wrote:
> Hey y'all,
>
> Please review the follow
Thanks Dong.
> 1. Currently if there is topic created with "." in the topic name, would
it cause correctness issue for this KIP?
Yes, RemoteClusterUtils would be confused by existing topics that have a
period, and MM2 might try to send records to existing topics if they happen
to be prefixed
+1 thx
On Mon, Jan 14, 2019, 8:06 AM Gwen Shapira +1. Thanks, that will be very helpful.
>
> On Mon, Jan 14, 2019, 4:43 AM Alex D
> > Hello guys,
> >
> > We need your VOTES for the KIP-379: Multiple Consumer Group Management.
> >
> > KIP-379:
> >
> >
>
Hey y'all,
Please review the following small KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-414%3A+Notify+SourceTask+of+ACK%27d+offsets%2C+metadata
Thanks!
Ryanne
Ryanne Dolan created KAFKA-7815:
---
Summary: SourceTask should expose ACK'd offsets, metadata
Key: KAFKA-7815
URL: https://issues.apache.org/jira/browse/KAFKA-7815
Project: Kafka
Issue Type
s a blocker for this KIP. So I am fine
> if people think we should defer the improvements of cross-cluster failover
> to a later decision.
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Tue, Jan 8, 2019 at 2:23 AM Ryanne Dolan wrote:
>
> > Hi Ewen, thanks fo
two eases this KIP's
> > implementation and user's implementation of AlterConfigPolicy (e.g deny
> all
> > requests that try to alter log level) significantly. We would also be
> able
> > to introduce a
> >
> > On Wed, Jan 9, 2019 at 1:48 AM Ryanne Dolan
> wrote
know the connection string to
> cluster B, which targetConsumerConfig provides.
>
> 107. Thanks. Could you add that description to the KIP?
>
> Thanks,
>
> Jun
>
> On Mon, Jan 7, 2019 at 3:50 PM Ryanne Dolan wrote:
>
>> Thanks Jun, I've updated the KIP as requested. Brief
> To differentiate between the normal Kafka config settings and the
application's log level settings, we will introduce a new resource type -
BROKER_LOGGERS
Stanislav, can you explain why log level wouldn't be a "normal Kafka config
setting"?
Ryanne
On Tue, Jan 8, 2019, 4:26 PM Stanislav
is that it will start as many as
> separate DistributedHerder as the Kafka clusters? So, essentially it's
> running multiple logical connect clusters on the same shared worker nodes?
>
> Thanks,
>
> Jun
>
>
> On Thu, Dec 20, 2018 at 5:23 PM Srinivas Reddy >
> wr
I'd also prefer to avoid the new configuration property if possible. Seems
like a lighter touch without it.
Ryanne
On Sun, Jan 6, 2019 at 7:25 PM Paul Davidson
wrote:
> Hi Konstantine,
>
> Thanks for your feedback! I think my reply to Ewen covers most of your
> points, and I mostly agree. If
ke MessageHandlers? I'm not sure I entirely understand the compatibility
> story here (which could also be that we just don't provide one -- just want
> to make sure it is clear).
>
> I may have missed something in this proposal since it's pretty long, let me
> know if there was somethi
t; +1 (binding). Nice work Ryan.
> > >>> -Harsha
> > >>>
> > >>> On Fri, Dec 21, 2018, at 8:14 AM, Andrew Schofield wrote:
> > >>>> +1 (non-binding)
> > >>>>
> > >>>> Andrew Schofiel
opic. (BTW, I
> assume that MM2 will mirror the timestamps from source to target without
> being changed)
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Thu, Dec 27, 2018 at 1:16 AM Ryanne Dolan
> wrote:
>
> > Becket, this is great feedback, thanks.
> >
> &g
o larger than the committed source offset. (e.g. committed
> offsets 150 will be mapped to the entry (99 -> 199))
> 2. Add a offsets difference because we know since that entry the offsets
> are increasing one by one. (target offsets = 199 + (150 -99) = 250)
>
> If the target offset is
cluster/herder/worker features" in a
> different KIP at some time? If yes, please feel free to let me know if
> I can provide any help on that front. Otherwise, I am also happy to
> draft a proposal as basis for discussion.
>
> Best regards,
> Sönke
>
> On Fri, De
Jun, let's leave the REST API out of the KIP then.
I have been arguing that Connect wouldn't benefit from the
multi-cluster/herder/worker features we need in MM2, and that the effort
would result in a needlessly complex Connect REST API. But certainly two
separate APIs is inherently more complex
> > > +1
> > >
> > > This looks like a huge project! Wikimedia would be very excited to have
> > > this. Thanks!
> > >
> > > On Thu, Dec 20, 2018 at 9:52 AM Ryanne Dolan
> > > wrote:
> > >
> > >
Hey y'all, please vote to adopt KIP-382 by replying +1 to this thread.
For your reference, here are the highlights of the proposal:
- Leverages the Kafka Connect framework and ecosystem.
- Includes both source and sink connectors.
- Includes a high-level driver that manages connectors in a
anded upon.
>
> Not sure if that has any actual impact on MM2 but wanted to at least
> mention it.
>
> Best regards,
> Sönke
>
> On Wed, Dec 19, 2018 at 11:00 PM Ryanne Dolan
> wrote:
>
> > Becket, thanks for taking a look.
> >
> > > 1. Only relying on the to
consumer may commit at arbitrary offset.
> Does the connector need to keep a mapping between each source offset to
> destination offset? If so how would that be done?
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Thu, Dec 13, 2018 at 8:23 AM Ryanne Dolan
> wrote:
>
&g
> Jun
>
> On Fri, Dec 14, 2018 at 1:25 PM Ryanne Dolan
> wrote:
>
> > Thanks Sönke, you're spot-on. I don't want MM2 to wait for Connect
> features
> > that don't exist yet, especially if MM2 is the primary use case for them.
> > Moreover, I think MM2 can drive a
single connect cluster model, yes, the co-existence of a
> MM2
> > REST API and the nearly identical Connect API is one of my concerns.
> > Implementation wise, my understanding is that the producer URL in a
> > SourceTask is always obtained from the connect worker's configuration.
>
> see B1,B2, A1,E1, A2, A3, E2 as the ordering by of the updates account is
> preserved.
>
> With the topic solution your suggesting we would have no way true way of
> replaying and re-constituting the order between X.account_state and
> Z.account_state topics in the case of E, as
issues where transactional processing is master-master in
> regions, where the processing is sticky to region but of failure or
> planned, processing of certain accounts move regions.
>
> Also I ask that you keep compatibility of the handler api interface in MM
> into MM2.
>
>
&
?
> >
> > One based on hops using headers, and another based on topic naming. After
> > all flexibility is what we want its for end users how to use right?
> >
> >
> >
> > On 12/7/18, 8:19 PM, "Ryanne Dolan" wrote:
> >
> > Michael
ichael Pearce
> wrote:
>
> > So this is indeed what using headers with hops avoids is creating lots
> and
> > lots of topics __, so you can have more complex topology setups.
> >
> > I ask why not support having two ways of setting up and closing the door?
> >
t; you document the public APIs?
>
> 4. source.cluster.bootstrap.servers/target.cluster.bootstrap.servers: Does
> a Source/Sink connect need both? Currently, the producer URL used in a
> SourceWorker always comes from the Worker configuration. Are you proposing
> to change
add handlers to have a little extra custom logic if needed, and
> the handler api is public today so should be supported going forwards so
> people are not on mass re-writing these.
>
> On 12/5/18, 5:36 PM, "Ryanne Dolan" wrote:
>
> Sönke,
>
>
th this before, maybe
> you'd like to share the burden :)
>
> Best regards,
> Sönke
>
> On Wed, Dec 5, 2018 at 5:15 AM Ryanne Dolan wrote:
>
> > Sönke,
> >
> > I think so long as we can keep the differences at a very high level (i.e.
> > the "
> to a certain extent this is down to a question of personal
> style/preference. And as this is your baby and you have put a lot more
> effort and thought into it than I ever will I'll shut up now :)
>
> Again, thanks for all your good work!
>
> Best regards,
> Sönke
>
ess implementation and maintenance effort.
>
> But again, all of that is based on my, potentially flawed, understanding of
> your proposal, please feel free to correct me :)
>
> Best regards,
> Sönke
>
> On Fri, Nov 30, 2018 at 1:39 AM Ryanne Dolan
> wrote:
>
> > S
er - I'd much prefer an option to use a
> regular connect cluster from an ops point of view. Is it maybe worth
> spending some time investigating whether we can come up with a change to
> connect that enables what MM would need?
>
> Best regards,
> Sönke
>
>
>
>
> > > overridden by connectors or if there were multiple places to
> specify
> > > what
> > > > > serde to use.
> > > >
> > > > There are multiple places that converters can be defined: the worker
> > > config
> > > > de
can specify topic whitelists and other connector-level settings here
too, or you can use the REST API to remote-control a running cluster.
I've also updated the KIP with minor changes to bring it in line with the
current implementation.
Looking forward to your feedback, thanks!
Ryanne
On Mon, Nov 19,
t; Interesting I was going to support topic messages with extra headers of
> source DC info, for cycle detection…..
>
> Looking forward your reply.
>
> Regards,
>
> Dan
> On 2018/10/23 19:56:02, Ryanne Dolan wrote:
> > Alex, thanks for the feedback.
> >
>
Stupid question: do you have transforms=DateConvert as well?
On Fri, Nov 9, 2018, 9:00 PM sanjeev0915 wrote:
> Hi
>
> Please help for the below issue
>
> i am using Kafka connect JDBC Connector and trying to pull the data from
> Oracle Database table. I am using Apache Kafka (not the
ot;cluster-name-foo") so in case another MM2 instance sees this message
> and it is configured to replicate data into "cluster-name-foo" it would
> just skip it instead of replicating it back.
>
> On Sat, Oct 20, 2018 at 5:48 AM Ryanne Dolan
> wrote:
>
> &
Thanks Harsha. Done.
On Fri, Oct 19, 2018 at 1:03 AM Harsha Chintalapani wrote:
> Ryanne,
>Makes sense. Can you please add this under rejected alternatives so
> that everyone has context on why it wasn’t picked.
>
> Thanks,
> Harsha
> On Oct 18, 2018, 8:02 A
, coordination etc. This also means that existing tooling,
dashboards etc that work with Connectors do not work with uReplicator, and
any future tooling would need to treat uReplicator as a special case.
Ryanne
On Wed, Oct 17, 2018 at 12:30 PM Ryanne Dolan wrote:
> Harsha, yes I can
, which the present
API exposes.
That said, I'm not opposed to your proposed callbacks, and I agree that
commit() and commitRecord() are poorly named. I just don't believe the
present API is incorrect.
Ryanne
On Thu, Oct 18, 2018 at 7:04 AM Per Steffensen wrote:
> On 17/10/2018 18.17, Rya
ng away from this another step.
>
> On 17.10.2018 17:34, Ryanne Dolan wrote:
> > Jan, these are two separate issues.
> >
> > 1) consumer coordination should not, ideally, involve unreliable or slow
> > connections. Naively, a KafkaSourceConnector would coordinate via the
>
Zahari, that makes sense, thanks for reframing your question. I suspect
that pause/resume was not intended to be called at high frequency like
that, but I agree with you that the current behavior is needlessly
inefficient. I like your idea of making it configurable.
Ryanne
On Thu, Oct 18, 2018,
o understand what
> can be done on our end to improve the Akka streams integration.. Thanks in
> advance :)
>
> Zahari
>
> On Wed, Oct 17, 2018 at 5:49 PM Ryanne Dolan
> wrote:
>
> > Zahari,
> >
> > It sounds to me like this problem is due t
made you to consider a
> new design from ground up.
>
> Thanks,
> Harsha
>
> On Wed, Oct 17, 2018, at 8:34 AM, Ryanne Dolan wrote:
> > Jan, these are two separate issues.
> >
> > 1) consumer coordination should not, ideally, involve unreliable or slow
> > co
but I don't
think there's anything fundamentally missing from the API.
Ryanne
On Wed, Oct 17, 2018 at 10:24 AM Per Steffensen wrote:
> On 17/10/2018 16.43, Ryanne Dolan wrote:
> > I see, thanks.
> > On the other hand, the commitRecord() callback provides the functionality
> &g
t;
> Pretty sure without this _design choice_ you can skip on that exactly
> once already
>
> Best Jan
>
> On 16.10.2018 18:16, Ryanne Dolan wrote:
> > > But one big obstacle in this was
> > always that group coordination happened on the source cluster.
>
Zahari,
It sounds to me like this problem is due to Akka attempting to implement
additional backpressure on top of the Consumer API. I'd suggest they not do
that, and then this problem goes away.
Ryanne
On Wed, Oct 17, 2018 at 7:35 AM Zahari Dichev
wrote:
> Hi there,
>
> Are there any
hat commit() is given (via argument) a list/collection of
> the records for which it is a guarantee. Thats what my current fix does
> (see PR).
>
>
> On 16/10/2018 19.33, Ryanne Dolan wrote:
>
> Steff,
>
> > Guess people have used it, assuming that all records that h
"cycle detection" -- cyclical replication does not result in infinite
> > recursion.
>
> Oh - got it, it checks the entire prefix, which seems obvious to me in
> retrospect :)
>
> Rhys
>
>
> > On Oct 15, 2018, at 3:18 PM, Ryanne Dolan wrote:
> >
> >
Konstantine, thanks for the explanation, makes sense.
Ryanne
On Tue, Oct 16, 2018, 1:51 PM Konstantine Karantasis <
konstant...@confluent.io> wrote:
> Matthias, Ryanne, Rhys, Guozhang, thank you all for your comments!
>
> Ryanne, to try to address your specific comments, let me start by saying
Steff,
> Guess people have used it, assuming that all records that have been
polled > at the time of callback to "commit", have also had their offsets
committed. > But that is not true.
(excerpt from KIP)
The documentation for SourceTask.commit() reads:
> Commit the offsets, up to the offsets
esforce.com/open-sourcing-mirus-3ec2c8a38537). I
> strongly believe Mirrormaker itself needs an upgrade, so I'm not
> questioning that, but more on the technical side of the solution.
>
> Thanks
> Eno
>
> On Mon, Oct 15, 2018 at 11:19 PM Ryanne Dolan
> wrote:
>
> &g
e
> right?
>
> Other than that. It feels like the KIP has too much features where many
> of them are not really wanted and counter productive but I will just
> wait and see how the discussion goes.
>
> Best Jan
>
>
> On 15.10.2018 18:16, Ryanne Dolan wrote:
> > Hey
;
> Excited to see the discussion on this one.
>
> Rhys
>
> > On Oct 15, 2018, at 9:16 AM, Ryanne Dolan wrote:
> >
> > Hey y'all!
> >
> > Please take a look at KIP-382:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0
> >
> > Thanks for your feedback and support.
> >
> > Ryanne
>
>
Hey y'all!
Please take a look at KIP-382:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0
Thanks for your feedback and support.
Ryanne
Ryanne Dolan created KAFKA-7500:
---
Summary: MirrorMaker 2.0 (KIP-382)
Key: KAFKA-7500
URL: https://issues.apache.org/jira/browse/KAFKA-7500
Project: Kafka
Issue Type: New Feature
Bravo!
On Thu, Oct 11, 2018 at 1:48 PM Ismael Juma wrote:
> Congratulations Manikumar! Thanks for your continued contributions.
>
> Ismael
>
> On Thu, Oct 11, 2018 at 10:39 AM Jason Gustafson
> wrote:
>
> > Hi all,
> >
> > The PMC for Apache Kafka has invited Manikumar Reddy as a committer and
[
https://issues.apache.org/jira/browse/KAFKA-6990?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Ryanne Dolan resolved KAFKA-6990.
-
Resolution: Not A Bug
> CommitFailedException; this task may be no longer owned by the thr
Konstantine, this is exciting work! Couple questions:
I understand that, overall, rebalances would require less work and less
time acquiring and releasing resources. But OTOH I believe individual
consumers might see successive revokes/assigns while a rebalance settles.
Is that right? And if so,
thanks!!
On Sun, Sep 30, 2018, 1:23 PM Matthias J. Sax wrote:
> Done
>
> On 9/27/18 11:51 AM, Ryanne Dolan wrote:
> > I would like to create some KIPs.
> >
> > JIRA user: ryannedolan
> > wiki user: ryannedolan
> > Ryanne Dolan
> > ryannedo...@gmail.com
> >
> > Thanks!
> > Ryanne
> >
>
>
I would like to create some KIPs.
JIRA user: ryannedolan
wiki user: ryannedolan
Ryanne Dolan
ryannedo...@gmail.com
Thanks!
Ryanne
Ryanne Dolan created KAFKA-7444:
---
Summary: expose connector, task IDs to SinkTasks
Key: KAFKA-7444
URL: https://issues.apache.org/jira/browse/KAFKA-7444
Project: Kafka
Issue Type: Improvement
he settings of the topics
> that will be created by Connect for that connector, which works with all
> existing source connectors out of the box and does not add additional
> complexities for source connector developers.
>
> Best regards,
>
> Randall
>
>
>
> On
is a chance the
> connector preference won't be used, connectors will have to force it via
> admin client which brings us back to the terrible config situation we
> currently have with Admin client.
>
> Gwen
>
>
> On Tue, Sep 11, 2018 at 7:23 PM, Ryanne Dolan
> wrote:
>
>
Randall,
I have some concerns with this proposal.
Firstly, I don't believe it is the job of a connector to configure topics,
generally, nor for topic-specific settings to hang out in connector
configurations. Automatic creation of topics with default settings is an
established pattern elsewhere,
201 - 282 of 282 matches
Mail list logo