You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@edgent.apache.org by Christofer Dutz <ch...@c-ware.de> on 2018/03/20 12:51:22 UTC

Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Hi,

have been using the Kafka integration quite often in the past and one thing I always have to explain when demonstrating code and which seems to confuse everyone seeing the code:

I would expect a KafkaConsumer to consume Edgent messages and publish them to Kafka and would expect a KafkaProducer to produce Edgent events.

Unfortunately it seems to be the other way around. This seems a little unintuitive. Judging from the continued confusion when demonstrating code eventually it’s worth considering to rename these (swap their names). Eventually even rename them to “KafkaSource” (Edgent Source that consumes Kafka messages and produces Edgent events) and “KafkaConsumer” (Consumes Edgent Events and produces Kafka messages). After all the Classes are in the Edgent namespace and come from the Edgent libs, so the fixed point when inspecting these should be clear. Also I bet no one would be confused if we called something that produces Kafka messages a consumer as there should never be code that handles this from a Kafka point of view AND uses Edgent at the same time.

Chris



Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by vino yang <ya...@gmail.com>.
Hi guys,

I also agree change the connectors renaming. When I wrote the RabbitMQ
connector, I watched the Kafka connector's implementation. There are two
communication pair : KafkaPublisher / KafkaSubscriber and KafkaConsumer /
KafkaProducer. I feel confused about them.

I think this mode would be better :


   - inside API : we should use Kafka client API to interact with Kafka
   Server , it's kafka's point of view, we could use KafkaProducer /
   KafkaConsumer;
   - outer API : it's edgent's point of view, we should use a unified
   naming, the name could be source / sink or some other edgent self-created
   named all connectors should follow this specification.


Vino yang
Thanks.

2018-03-23 0:56 GMT+08:00 Christofer Dutz <ch...@c-ware.de>:

> Hi Dale,
>
> Happy to read from you :-)
>
> It was just something I had to explain every time I showed the code for
> the currently by far most interesting use-case for my plc4x pocs at the
> moment (pumping data from a PLC to a Kafka topic) . So I thought, that if I
> have to explain it every time, cause people are confused, then probably we
> should talk about making things more clear.
>
> Chris
>
> Outlook for Android<https://aka.ms/ghei36> herunterladen
>
> ________________________________
> From: Dale LaBossiere <dm...@gmail.com>
> Sent: Thursday, March 22, 2018 5:44:42 PM
> To: dev@edgent.apache.org
> Subject: Re: Anyone else mis-interpret the "KafkaConsumer" and
> "KafkaProducer" all the time?
>
> A bit of background…
>
> The Kafka connector is two classes instead of a single KafkaStreams
> connector (with publish(),subscribe()) because at least a while ago, don’t
> know if this is still the case, Kafka had two completely separate classes
> for a “consumer” and a “producer" each with very different config setup
> params. By comparison MQTT has a single MqttClient class (with
> publish()/subscribe()).
>
> At the time, the decision was to name the Edgent Kafka classes similar to
> the underlying Kafka API classes.  Hence KafkaConsumer (~wrapping Kafka’s
> ConsumerConnector) and KafkaProducer (~wrapping Kafka’s KafkaProducer).
> While not exposed today, it’s conceivable that some day one could create an
> Edgent Kafka connector instance by providing a Kafka API class directly
> instead of just a config map - e.g., supplying a Kafka KafkaProducer as an
> arg to the Edgent KafkaProducer connector's constructor.  So having the
> names align seems like goodness.
>
> I don’t think the Edgent connectors should be trying to make it
> unnecessary for a user to understand or to mask the underlying system’s
> API… just make it usable, easily usable for a simple/common cases, in an
> Edgent topology context (worrying about when to make an actually external
> connection, recovering from broken connections / reconnecting, handling
> common tuple types).
>
> As for the specific suggestions, I think simply switching the names of
> Edgent’s KafkaConsumer and KafkaProducer is a bad idea :-)
>
> Offering KafkaSource and KafkaSink is OK I guess (though probably
> retaining the current names for a release or three).  Though I’ll note the
> Edgent API uses “source” and “sink” as verbs, which take a Supplier and a
> Consumer fn as args respectively.  Note Consumer used in the context with
> sink.
>
> Alternatively there’s KafkaSubscriber and KafkaPublisher.  While clearer
> than Consumer/Producer, I don’t know if they’re any better than Source/Sink.
>
> In the end I guess I don’t feel strongly about it all… though wonder if
> it’s really worth the effort in changing.  At least the Edgent connector’s
> javadoc is pretty good / clear for the classes and their use... I think :-)
>
> — Dale
>
>
> > On Mar 20, 2018, at 9:59 PM, vino yang <ya...@gmail.com> wrote:
> >
> > Hi Chris,
> >
> > All data processing framework could think it as a *pipeline . *The
> Edgent's
> > point of view, there could be two endpoints :
> >
> >
> >   - source : means data injection;
> >   - sink : means data export;
> >
> > There are many frameworks use this conventional naming rule, such as
> Apache
> > Flume, Apache Flink, Apache Spark(structured streaming) .
> >
> > I think "KafkaConsumer" could be replaced with "KafkaSource" and
> > "KafkaProducer" could be named "KafkaSink".
> >
> > And middle of the pipeline is the transformation of the data, there are
> > many operators to transform data ,such as map, flatmap, filter, reduce...
> > and so on.
> >
> > Vino yang.
> > Thanks.
> >
> > 2018-03-20 20:51 GMT+08:00 Christofer Dutz <ch...@c-ware.de>:
> >
> >> Hi,
> >>
> >> have been using the Kafka integration quite often in the past and one
> >> thing I always have to explain when demonstrating code and which seems
> to
> >> confuse everyone seeing the code:
> >>
> >> I would expect a KafkaConsumer to consume Edgent messages and publish
> them
> >> to Kafka and would expect a KafkaProducer to produce Edgent events.
> >>
> >> Unfortunately it seems to be the other way around. This seems a little
> >> unintuitive. Judging from the continued confusion when demonstrating
> code
> >> eventually it’s worth considering to rename these (swap their names).
> >> Eventually even rename them to “KafkaSource” (Edgent Source that
> consumes
> >> Kafka messages and produces Edgent events) and “KafkaConsumer” (Consumes
> >> Edgent Events and produces Kafka messages). After all the Classes are in
> >> the Edgent namespace and come from the Edgent libs, so the fixed point
> when
> >> inspecting these should be clear. Also I bet no one would be confused
> if we
> >> called something that produces Kafka messages a consumer as there should
> >> never be code that handles this from a Kafka point of view AND uses
> Edgent
> >> at the same time.
> >>
> >> Chris
> >>
> >>
> >>
>
>

Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by vino yang <ya...@gmail.com>.
Hi Dale,

The producer and consumer do not share one connection, just avoid they are
both used in one topology (if we share the connection, the producer and
consumer which close the connection would trigger another failure).
Actually, this case I just described rarely occurs(that means the most of
the scenario there is only one connection) . So I think we should take it
easy about the case you described.

Vino yang.
Thanks.

2018-03-27 4:27 GMT+08:00 Dale LaBossiere <dm...@gmail.com>:

> Hi Vino, thanks for the clarification.
>
> One last question :-). Is there ever a situation when it’s
> desirable/possible for a Producer and Consumer to share a single RabbetMQ
> connection?  e.g., a low throughput device wanting to minimize
> connections?  If so, the separate Producer and Consumer split doesn't
> support that case.
>
> > On Mar 22, 2018, at 9:39 PM, vino yang <ya...@gmail.com> wrote:
> >
> > Hi Dale,
> >
> > When I wroted the RabbitMQ connector I followed the Kafka Connector's
> style
> > (and I also looked the MQTT connectors). And I chose the Kafka connector
> as
> > the implementation template. The reason is the two classes
> > (RabbitmqProducer and RabbitmqConsumer) should not share one rabbitmq's
> > connection and channel (implemented in RabbitmqConnector). The two
> classes
> > maybe use in one topology (as consumer and producer) and split the inner
> > connection and channel would be better.
> >
> > 2018-03-23 2:28 GMT+08:00 Dale LaBossiere <dl...@apache.org>:
> >
> >> I see the new RabbitMQ connector followed the same API scheme as the
> Kafka
> >> connector.  i.e., adding Rabbitmq{Consumer,Producer} for the source/sink
> >> respectively.  It looks like it could have followed the MqttStreams
> >> approach instead.
> >>
> >> @yanghua, is there a reason you chose to offer
> o.a.e.connectors.rabbitmq.Rabbitmq{Consumer,Producer}
> >> instead of just RabbitmqStreams?
> >>
> >> — Dale
> >>
> >>> On Mar 22, 2018, at 1:11 PM, Dale LaBossiere <dm...@gmail.com>
> >> wrote:
> >>>
> >>> Hi Chris.  Hopefully the background provided some useful context.  But
> >> like I said, I don’t feel strongly about some renaming if folks agree
> >> that’s the right think to do.
> >>>
> >>> — Dale
> >>>
> >>>> On Mar 22, 2018, at 12:56 PM, Christofer Dutz <
> >> christofer.dutz@c-ware.de> wrote:
> >>>> It was just something I had to explain every time I showed the code
> for
> >> the currently by far most interesting use-case for my plc4x pocs at the
> >> moment (pumping data from a PLC to a Kafka topic) . So I thought, that
> if I
> >> have to explain it every time, cause people are confused, then probably
> we
> >> should talk about making things more clear.
> >>>
> >>
> >>
>
>

Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by Dale LaBossiere <dm...@gmail.com>.
Hi Vino, thanks for the clarification.

One last question :-). Is there ever a situation when it’s desirable/possible for a Producer and Consumer to share a single RabbetMQ connection?  e.g., a low throughput device wanting to minimize connections?  If so, the separate Producer and Consumer split doesn't support that case.

> On Mar 22, 2018, at 9:39 PM, vino yang <ya...@gmail.com> wrote:
> 
> Hi Dale,
> 
> When I wroted the RabbitMQ connector I followed the Kafka Connector's style
> (and I also looked the MQTT connectors). And I chose the Kafka connector as
> the implementation template. The reason is the two classes
> (RabbitmqProducer and RabbitmqConsumer) should not share one rabbitmq's
> connection and channel (implemented in RabbitmqConnector). The two classes
> maybe use in one topology (as consumer and producer) and split the inner
> connection and channel would be better.
> 
> 2018-03-23 2:28 GMT+08:00 Dale LaBossiere <dl...@apache.org>:
> 
>> I see the new RabbitMQ connector followed the same API scheme as the Kafka
>> connector.  i.e., adding Rabbitmq{Consumer,Producer} for the source/sink
>> respectively.  It looks like it could have followed the MqttStreams
>> approach instead.
>> 
>> @yanghua, is there a reason you chose to offer o.a.e.connectors.rabbitmq.Rabbitmq{Consumer,Producer}
>> instead of just RabbitmqStreams?
>> 
>> — Dale
>> 
>>> On Mar 22, 2018, at 1:11 PM, Dale LaBossiere <dm...@gmail.com>
>> wrote:
>>> 
>>> Hi Chris.  Hopefully the background provided some useful context.  But
>> like I said, I don’t feel strongly about some renaming if folks agree
>> that’s the right think to do.
>>> 
>>> — Dale
>>> 
>>>> On Mar 22, 2018, at 12:56 PM, Christofer Dutz <
>> christofer.dutz@c-ware.de> wrote:
>>>> It was just something I had to explain every time I showed the code for
>> the currently by far most interesting use-case for my plc4x pocs at the
>> moment (pumping data from a PLC to a Kafka topic) . So I thought, that if I
>> have to explain it every time, cause people are confused, then probably we
>> should talk about making things more clear.
>>> 
>> 
>> 


Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by vino yang <ya...@gmail.com>.
Hi Dale,

When I wroted the RabbitMQ connector I followed the Kafka Connector's style
(and I also looked the MQTT connectors). And I chose the Kafka connector as
the implementation template. The reason is the two classes
(RabbitmqProducer and RabbitmqConsumer) should not share one rabbitmq's
connection and channel (implemented in RabbitmqConnector). The two classes
maybe use in one topology (as consumer and producer) and split the inner
connection and channel would be better.

2018-03-23 2:28 GMT+08:00 Dale LaBossiere <dl...@apache.org>:

> I see the new RabbitMQ connector followed the same API scheme as the Kafka
> connector.  i.e., adding Rabbitmq{Consumer,Producer} for the source/sink
> respectively.  It looks like it could have followed the MqttStreams
> approach instead.
>
> @yanghua, is there a reason you chose to offer o.a.e.connectors.rabbitmq.Rabbitmq{Consumer,Producer}
> instead of just RabbitmqStreams?
>
> — Dale
>
> > On Mar 22, 2018, at 1:11 PM, Dale LaBossiere <dm...@gmail.com>
> wrote:
> >
> > Hi Chris.  Hopefully the background provided some useful context.  But
> like I said, I don’t feel strongly about some renaming if folks agree
> that’s the right think to do.
> >
> > — Dale
> >
> >> On Mar 22, 2018, at 12:56 PM, Christofer Dutz <
> christofer.dutz@c-ware.de> wrote:
> >> It was just something I had to explain every time I showed the code for
> the currently by far most interesting use-case for my plc4x pocs at the
> moment (pumping data from a PLC to a Kafka topic) . So I thought, that if I
> have to explain it every time, cause people are confused, then probably we
> should talk about making things more clear.
> >
>
>

Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by Dale LaBossiere <dl...@apache.org>.
I see the new RabbitMQ connector followed the same API scheme as the Kafka connector.  i.e., adding Rabbitmq{Consumer,Producer} for the source/sink respectively.  It looks like it could have followed the MqttStreams approach instead.

@yanghua, is there a reason you chose to offer o.a.e.connectors.rabbitmq.Rabbitmq{Consumer,Producer} instead of just RabbitmqStreams?

— Dale

> On Mar 22, 2018, at 1:11 PM, Dale LaBossiere <dm...@gmail.com> wrote:
> 
> Hi Chris.  Hopefully the background provided some useful context.  But like I said, I don’t feel strongly about some renaming if folks agree that’s the right think to do.
> 
> — Dale
> 
>> On Mar 22, 2018, at 12:56 PM, Christofer Dutz <ch...@c-ware.de> wrote:
>> It was just something I had to explain every time I showed the code for the currently by far most interesting use-case for my plc4x pocs at the moment (pumping data from a PLC to a Kafka topic) . So I thought, that if I have to explain it every time, cause people are confused, then probably we should talk about making things more clear.
> 


Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by Dale LaBossiere <dm...@gmail.com>.
Hi Chris.  Hopefully the background provided some useful context.  But like I said, I don’t feel strongly about some renaming if folks agree that’s the right think to do.

— Dale

> On Mar 22, 2018, at 12:56 PM, Christofer Dutz <ch...@c-ware.de> wrote:
> It was just something I had to explain every time I showed the code for the currently by far most interesting use-case for my plc4x pocs at the moment (pumping data from a PLC to a Kafka topic) . So I thought, that if I have to explain it every time, cause people are confused, then probably we should talk about making things more clear.


Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by Christofer Dutz <ch...@c-ware.de>.
Hi Dale,

Happy to read from you :-)

It was just something I had to explain every time I showed the code for the currently by far most interesting use-case for my plc4x pocs at the moment (pumping data from a PLC to a Kafka topic) . So I thought, that if I have to explain it every time, cause people are confused, then probably we should talk about making things more clear.

Chris

Outlook for Android<https://aka.ms/ghei36> herunterladen

________________________________
From: Dale LaBossiere <dm...@gmail.com>
Sent: Thursday, March 22, 2018 5:44:42 PM
To: dev@edgent.apache.org
Subject: Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

A bit of background…

The Kafka connector is two classes instead of a single KafkaStreams connector (with publish(),subscribe()) because at least a while ago, don’t know if this is still the case, Kafka had two completely separate classes for a “consumer” and a “producer" each with very different config setup params. By comparison MQTT has a single MqttClient class (with publish()/subscribe()).

At the time, the decision was to name the Edgent Kafka classes similar to the underlying Kafka API classes.  Hence KafkaConsumer (~wrapping Kafka’s ConsumerConnector) and KafkaProducer (~wrapping Kafka’s KafkaProducer).  While not exposed today, it’s conceivable that some day one could create an Edgent Kafka connector instance by providing a Kafka API class directly instead of just a config map - e.g., supplying a Kafka KafkaProducer as an arg to the Edgent KafkaProducer connector's constructor.  So having the names align seems like goodness.

I don’t think the Edgent connectors should be trying to make it unnecessary for a user to understand or to mask the underlying system’s API… just make it usable, easily usable for a simple/common cases, in an Edgent topology context (worrying about when to make an actually external connection, recovering from broken connections / reconnecting, handling common tuple types).

As for the specific suggestions, I think simply switching the names of Edgent’s KafkaConsumer and KafkaProducer is a bad idea :-)

Offering KafkaSource and KafkaSink is OK I guess (though probably retaining the current names for a release or three).  Though I’ll note the Edgent API uses “source” and “sink” as verbs, which take a Supplier and a Consumer fn as args respectively.  Note Consumer used in the context with sink.

Alternatively there’s KafkaSubscriber and KafkaPublisher.  While clearer than Consumer/Producer, I don’t know if they’re any better than Source/Sink.

In the end I guess I don’t feel strongly about it all… though wonder if it’s really worth the effort in changing.  At least the Edgent connector’s javadoc is pretty good / clear for the classes and their use... I think :-)

— Dale


> On Mar 20, 2018, at 9:59 PM, vino yang <ya...@gmail.com> wrote:
>
> Hi Chris,
>
> All data processing framework could think it as a *pipeline . *The Edgent's
> point of view, there could be two endpoints :
>
>
>   - source : means data injection;
>   - sink : means data export;
>
> There are many frameworks use this conventional naming rule, such as Apache
> Flume, Apache Flink, Apache Spark(structured streaming) .
>
> I think "KafkaConsumer" could be replaced with "KafkaSource" and
> "KafkaProducer" could be named "KafkaSink".
>
> And middle of the pipeline is the transformation of the data, there are
> many operators to transform data ,such as map, flatmap, filter, reduce...
> and so on.
>
> Vino yang.
> Thanks.
>
> 2018-03-20 20:51 GMT+08:00 Christofer Dutz <ch...@c-ware.de>:
>
>> Hi,
>>
>> have been using the Kafka integration quite often in the past and one
>> thing I always have to explain when demonstrating code and which seems to
>> confuse everyone seeing the code:
>>
>> I would expect a KafkaConsumer to consume Edgent messages and publish them
>> to Kafka and would expect a KafkaProducer to produce Edgent events.
>>
>> Unfortunately it seems to be the other way around. This seems a little
>> unintuitive. Judging from the continued confusion when demonstrating code
>> eventually it’s worth considering to rename these (swap their names).
>> Eventually even rename them to “KafkaSource” (Edgent Source that consumes
>> Kafka messages and produces Edgent events) and “KafkaConsumer” (Consumes
>> Edgent Events and produces Kafka messages). After all the Classes are in
>> the Edgent namespace and come from the Edgent libs, so the fixed point when
>> inspecting these should be clear. Also I bet no one would be confused if we
>> called something that produces Kafka messages a consumer as there should
>> never be code that handles this from a Kafka point of view AND uses Edgent
>> at the same time.
>>
>> Chris
>>
>>
>>


Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by Dale LaBossiere <dm...@gmail.com>.
A bit of background…

The Kafka connector is two classes instead of a single KafkaStreams connector (with publish(),subscribe()) because at least a while ago, don’t know if this is still the case, Kafka had two completely separate classes for a “consumer” and a “producer" each with very different config setup params. By comparison MQTT has a single MqttClient class (with publish()/subscribe()).

At the time, the decision was to name the Edgent Kafka classes similar to the underlying Kafka API classes.  Hence KafkaConsumer (~wrapping Kafka’s ConsumerConnector) and KafkaProducer (~wrapping Kafka’s KafkaProducer).  While not exposed today, it’s conceivable that some day one could create an Edgent Kafka connector instance by providing a Kafka API class directly instead of just a config map - e.g., supplying a Kafka KafkaProducer as an arg to the Edgent KafkaProducer connector's constructor.  So having the names align seems like goodness.

I don’t think the Edgent connectors should be trying to make it unnecessary for a user to understand or to mask the underlying system’s API… just make it usable, easily usable for a simple/common cases, in an Edgent topology context (worrying about when to make an actually external connection, recovering from broken connections / reconnecting, handling common tuple types).

As for the specific suggestions, I think simply switching the names of Edgent’s KafkaConsumer and KafkaProducer is a bad idea :-)

Offering KafkaSource and KafkaSink is OK I guess (though probably retaining the current names for a release or three).  Though I’ll note the Edgent API uses “source” and “sink” as verbs, which take a Supplier and a Consumer fn as args respectively.  Note Consumer used in the context with sink.

Alternatively there’s KafkaSubscriber and KafkaPublisher.  While clearer than Consumer/Producer, I don’t know if they’re any better than Source/Sink.

In the end I guess I don’t feel strongly about it all… though wonder if it’s really worth the effort in changing.  At least the Edgent connector’s javadoc is pretty good / clear for the classes and their use... I think :-)

— Dale


> On Mar 20, 2018, at 9:59 PM, vino yang <ya...@gmail.com> wrote:
> 
> Hi Chris,
> 
> All data processing framework could think it as a *pipeline . *The Edgent's
> point of view, there could be two endpoints :
> 
> 
>   - source : means data injection;
>   - sink : means data export;
> 
> There are many frameworks use this conventional naming rule, such as Apache
> Flume, Apache Flink, Apache Spark(structured streaming) .
> 
> I think "KafkaConsumer" could be replaced with "KafkaSource" and
> "KafkaProducer" could be named "KafkaSink".
> 
> And middle of the pipeline is the transformation of the data, there are
> many operators to transform data ,such as map, flatmap, filter, reduce...
> and so on.
> 
> Vino yang.
> Thanks.
> 
> 2018-03-20 20:51 GMT+08:00 Christofer Dutz <ch...@c-ware.de>:
> 
>> Hi,
>> 
>> have been using the Kafka integration quite often in the past and one
>> thing I always have to explain when demonstrating code and which seems to
>> confuse everyone seeing the code:
>> 
>> I would expect a KafkaConsumer to consume Edgent messages and publish them
>> to Kafka and would expect a KafkaProducer to produce Edgent events.
>> 
>> Unfortunately it seems to be the other way around. This seems a little
>> unintuitive. Judging from the continued confusion when demonstrating code
>> eventually it’s worth considering to rename these (swap their names).
>> Eventually even rename them to “KafkaSource” (Edgent Source that consumes
>> Kafka messages and produces Edgent events) and “KafkaConsumer” (Consumes
>> Edgent Events and produces Kafka messages). After all the Classes are in
>> the Edgent namespace and come from the Edgent libs, so the fixed point when
>> inspecting these should be clear. Also I bet no one would be confused if we
>> called something that produces Kafka messages a consumer as there should
>> never be code that handles this from a Kafka point of view AND uses Edgent
>> at the same time.
>> 
>> Chris
>> 
>> 
>> 


Re: Anyone else mis-interpret the "KafkaConsumer" and "KafkaProducer" all the time?

Posted by vino yang <ya...@gmail.com>.
Hi Chris,

All data processing framework could think it as a *pipeline . *The Edgent's
point of view, there could be two endpoints :


   - source : means data injection;
   - sink : means data export;

There are many frameworks use this conventional naming rule, such as Apache
Flume, Apache Flink, Apache Spark(structured streaming) .

I think "KafkaConsumer" could be replaced with "KafkaSource" and
"KafkaProducer" could be named "KafkaSink".

And middle of the pipeline is the transformation of the data, there are
many operators to transform data ,such as map, flatmap, filter, reduce...
and so on.

Vino yang.
Thanks.

2018-03-20 20:51 GMT+08:00 Christofer Dutz <ch...@c-ware.de>:

> Hi,
>
> have been using the Kafka integration quite often in the past and one
> thing I always have to explain when demonstrating code and which seems to
> confuse everyone seeing the code:
>
> I would expect a KafkaConsumer to consume Edgent messages and publish them
> to Kafka and would expect a KafkaProducer to produce Edgent events.
>
> Unfortunately it seems to be the other way around. This seems a little
> unintuitive. Judging from the continued confusion when demonstrating code
> eventually it’s worth considering to rename these (swap their names).
> Eventually even rename them to “KafkaSource” (Edgent Source that consumes
> Kafka messages and produces Edgent events) and “KafkaConsumer” (Consumes
> Edgent Events and produces Kafka messages). After all the Classes are in
> the Edgent namespace and come from the Edgent libs, so the fixed point when
> inspecting these should be clear. Also I bet no one would be confused if we
> called something that produces Kafka messages a consumer as there should
> never be code that handles this from a Kafka point of view AND uses Edgent
> at the same time.
>
> Chris
>
>
>