Hi,

thanks for the feedback, you have some valid points that I did not consider
thoroughly.

Now that I think about it, my current proposal is indeed a bit hacky,
namely the dichotomy between running "light"/"heavy" connectors in
standalone/distributed mode: indeed there should be no difference, one
should be able to freely scale a connector by running more container
instances. Whereas with the proposed approach going from 1 container to
more would require switching the runtime mode, e.g. this is completely not
the case with kafka streams apps -- you just scale them at will and I don't
see why connectors should behave differently.

Your points regarding the config/status topics are also true: the config
topic seems irrelevant assuming connector configuration is defined
statically inside the container and the status topic could _probably_ also
be dropped and the functionality implemented via the consumer group
protocol -- though I'm also just speculating here, I'll have to study the
source code in more depth (maybe someone more familiar with the code-base
could offer some insight?).

Regarding Rahul's questions as to why these topics are a problem, I can
also re-iterate what Stephane said: it's a needless burden to manage.
Though I'm pretty sure different connectors can use the same topics, it's
still not nice, since if you're using ACLs to control access, all
connectors will have to be granted access to these topics and there can
potentially be trouble from misbehaving/malicious connectors. Also, I don't
think these topics offer any advantages when it comes to centralization:
the only reasons they exist is because the distributed mode has no other
way to store information apart from inside kafka topics, whereas if you're
running connectors in containers on kubernetes, you'd store this
configuration in the image/env-vars/configmaps or some other mechanism you
use, the point being that this once again becomes a concern for the
container platform -- once again, just like with Kafka Streams.

Given all of the above, I'm now inclined to either extend the standalone
mode and enable scaling/offset storage in kafka OR better yet, introduce a
new runtime mode ("container"? not sure what should it be called). The key
points of this runtime mode would be:

 - offset storage in a kafka topic
 - only one statically configured connector launched during startup
 - scaling happens via the consumer group protocol
 - only a lightweight "health" REST API that simply informs if the
connector is running

Obviously this would extend the scope of the KIP, but I'd be willing to
give this a shot.
Waiting for your feedback, once a more clear vision is in place I could
update the KIP.

Thanks

2018-05-17 13:17 GMT+03:00 Stephane Maarek <steph...@simplemachines.com.au>:

> Hi Salius
>
> I think you're on the money, but you're not pushing things too far.
> This is something I've hoped for a long time.
> Let's talk Kafka Connect v2
>
> Kafka Connect Cluster, as you said, are not convenient to work with (the
> KIP details drawbacks well). I'm all about containerisation just like
> stream apps support (and boasts!).
>
> Now, here's the problem with Kafka Connect. There are three backing topics.
> Here's the analysis of how they can evolve:
> - Config topic: this one is irrelevant if each connect cluster comes with a
> config bundled with the corresponding JAR, as you mentioned in your KIP
> - Status topic: this is something I wish was gone too. The consumers have a
> coordinator, and I believe the connect workers should have a coordinator
> too, for task rebalancing.
> - Source Offset topic: only relevant for sources. I wish there was a
> __connect_offsets global topic just like for consumers and an
> "ConnectOffsetCoordinator" to talk to to retrieve latest committed offset.
>
> If we look above, with a few back-end fundamental transformations, we can
> probably make Connect "cluster-less".
>
> What the community would get out of it is huge:
> - Connect workers for a specific connector are independent and isolated,
> measurable (in CPU and Mem) and auto-scalable
> - CI/CD is super easy to integrate, as it's just another container / jar.
> - You can roll restart a specific connector and upgrade a JAR without
> interrupting your other connectors and while keeping the current connector
> from running.
> - The topics backing connect are removed except the global one, which
> allows you to scale easily in terms of number of connectors
> - Running a connector in dev or prod (for people offering connectors) is as
> easy as doing a simple "docker run".
> - Each consumer / producer settings can be configured at the container
> level.
> - Each connect process is immutable in configuration.
> - Each connect process has its own security identity (right now, you need a
> connect cluster per service role, which is a lot of overhead in terms of
> backing topic)
>
> Now, I don't have the Kafka expertise to know exactly which changes to make
> in the code, but I believe the final idea is achievable.
> The change would be breaking for how Kafka Connect is run, but I think
> there's a chance to make the change non breaking to how Connect is
> programmed. I believe the same public API framework can be used.
>
> Finally, the REST API can be used for monitoring, or the JMX metrics as
> usual.
>
> I may be completely wrong, but I would see such a change drive the
> utilisation, management of Connect by a lot while lowering the barrier to
> adoption.
>
> This change may be big to implement but probably worthwhile. I'd be happy
> to provide more "user feedback" on a PR, but probably won't be able to
> implement a PR myself.
>
> More than happy to discuss this
>
> Best,
> Stephane
>
>
> Kind regards,
> Stephane
>
> [image: Simple Machines]
>
> Stephane Maarek | Developer
>
> +61 416 575 980
> steph...@simplemachines.com.au
> simplemachines.com.au
> Level 2, 145 William Street, Sydney NSW 2010
>
> On 17 May 2018 at 14:42, Saulius Valatka <saulius...@gmail.com> wrote:
>
> > Hi,
> >
> > the only real usecase for the REST interface I can see is providing
> > health/liveness checks for mesos/kubernetes. It's also true that the API
> > can be left as is and e.g. not exposed publicly on the platform level,
> but
> > this would still leave opportunities to accidentally mess something up
> > internally, so it's mostly a safety concern.
> >
> > Regarding the option renaming: I agree that it's not necessary as it's
> not
> > clashing with anything, my reasoning is that assuming some other offset
> > storage appears in the future, having all config properties at the root
> > level of offset.storage.* _MIGHT_ introduce clashes in the future, so
> this
> > is just a suggestion for introducing a convention of
> > offset.storage.<store>.<properties>, which the existing
> > property offset.storage.file.filename already adheres to. But in general,
> > yes -- this can be left as is.
> >
> >
> >
> > 2018-05-17 1:20 GMT+03:00 Jakub Scholz <ja...@scholz.cz>:
> >
> > > Hi,
> > >
> > > What do you plan to use the read-only REST interface for? Is there
> > > something what you cannot get through metrics interface? Otherwise it
> > might
> > > be easier to just disable the REST interface (either in the code, or
> just
> > > on the platform level - e.g. in Kubernetes).
> > >
> > > Also, I do not know what is the usual approach in Kafka ... but do we
> > > really have to rename the offset.storage.* options? The current names
> do
> > > not seem to have any collision with what you are adding and they would
> > get
> > > "out of sync" with the other options used in connect (status.storage.*
> > and
> > > config.storage.*). So it seems a bit unnecessary change to me.
> > >
> > > Jakub
> > >
> > >
> > >
> > > On Wed, May 16, 2018 at 10:10 PM Saulius Valatka <saulius...@gmail.com
> >
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > I'd like to start a discussion on the following KIP:
> > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > 304%3A+Connect+runtime+mode+improvements+for+container+platforms
> > > >
> > > > Basically the idea is to make it easier to run separate instances of
> > > Kafka
> > > > Connect hosting isolated connectors on container platforms such as
> > Mesos
> > > or
> > > > Kubernetes.
> > > >
> > > > In particular it would be interesting to hear opinions about the
> > proposed
> > > > read-only REST API mode, more specifically I'm concerned about the
> > > > possibility to implement it in distributed mode as it appears the
> > > framework
> > > > is using it internally (
> > > >
> > > > https://github.com/apache/kafka/blob/trunk/connect/
> > > runtime/src/main/java/org/apache/kafka/connect/runtime/
> > > distributed/DistributedHerder.java#L1019
> > > > ),
> > > > however this particular API method appears to be undocumented(?).
> > > >
> > > > Looking forward for your feedback.
> > > >
> > >
> >
>

Reply via email to