+1 (non-binding)
passes librdkafka integration test suite
Den fre 26 okt. 2018 kl 15:58 skrev Manikumar :
> minor observation: config sections are empty in the documentation page.
> http://kafka.apache.org/21/documentation.html#producerconfigs
>
> On Wed, Oct 24, 2018 at 10:49 PM Ted Yu wrote:
+1 (non-binding)
Passes librdkafka integration test suite
Den mån 29 okt. 2018 kl 18:08 skrev Manikumar :
> Hi Eno,
>
> This looks like an existing issue occuring only on source artifacts. We
> are able to generate aggregate docs on cloned repo.
> I am getting similar error on previous release a
Sorry for getting in the game this late, and on the wrong thread!
I think negative timestamps makes sense and is a good addition,
but I have a couple of concerns with the proposal:
1. I believe any change to the protocol format or semantics require a
protocol bump, in this case for ProduceReques
wing us to avoid even offering insecure algorithms
> during the TLS handshake. I.e: avoid the usage of certificate
> signature algorithms involving SHA1.
>
> Magnus Edenhill mentioned in PR[3] some ongoing work/discussion about
> keeping librdkafka capabilities inline with the corre
to the authorizer
> > we could have
> > >>> >>> > > >> >> Zookeeper> >> >> as a default though and a client that
> > manages SCRAM credentials
> > >>> >>> > > >> >> in ZK.> >> >&
+1 (non-binding)
2018-07-04 13:40 GMT+02:00 Satish Duggana :
> +1
>
> Thanks,
> Satish.
>
> On Wed, Jul 4, 2018 at 4:11 PM, Daniele Ascione
> wrote:
>
> > +1
> >
> > Thanks,
> > Daniele
> >
> > Il giorno mar 3 lug 2018 alle ore 23:55 Harsha ha
> > scritto:
> >
> > > +1.
> > >
> > > Thanks,
> >
Thanks for driving this KIP, Colin.
I agree with Dong that a new similar modifyConfigs API (and protocol API)
is confusing and that
we should try to extend the current alterConfigs interface to support the
incremental mode instead,
deprecating the non-incremental mode in the process.
Another thin
Hi Viktor,
since serialization of floats isn't as straight forward as integers, please
specify the exact serialization format of DOUBLE in the protocol docs
(e.g., IEEE 754),
including endianness (big-endian please).
This will help the non-java client ecosystem.
Thanks,
Magnus
2018-04-09 15:16
alizer
> > > > writes eight bytes containing the given double value, in Big Endian
> > byte
> > > > order, into this buffer at the current position, and then increments
> > the
> > > > position by eight.
> > > >
> > > > The implementa
> Ismael wrote:
> Jason, that's an interesting point regarding the Java client. Do we know
> what clients in other languages do in these cases?
librdkafka (and its bindings) passes unknown/future errors through to the
application, the error code remains intact while
the error string will be set to
+1
Passes librdkafka test suite.
Den fre 8 feb. 2019 kl 21:02 skrev Colin McCabe :
> Hi all,
>
> This is the third candidate for release of Apache Kafka 2.1.1. This
> release includes many bug fixes for Apache Kafka 2.1.
>
> Compared to rc1, this release includes the following changes:
> * MINO
Hey Rajini,
good work on the KIP!
I'm personally not thrilled with piggy-backing the authorized_operations on
existing APIs,
it seems like a mix of concerns.
Since there is already the notion of generic ResourceType,ResourceName
tuples in the Admin API, I believe it would make more sense to add a
+1 (non-binding)
Den ons 10 apr. 2019 kl 02:38 skrev Guozhang Wang :
> +1 (binding). Thanks for the written KIP! The approach lgtm.
>
> One minor thing: the name of "last epoch" maybe a bit misleading (although
> it is for internal usage only and will not be exposed to users) for future
> develo
For what it is worth:
librdkafka-based clients will treat any subscribed topic string that is
prefixed with the regexp start-of-line anchor ("^") as a regexp,
thus providing regex and actual topic subscriptions with the same API, also
allowing a mix of actuals and regexps:
E.g.:
consumer.subscribe
Bumping this thread so Wes can reply to it. Ignore this mail.
2016-02-24 0:36 GMT+01:00 Joel Koshy :
> Great - thanks for clarifying.
>
> Joel
>
> On Tue, Feb 23, 2016 at 1:47 PM, Bill Warshaw wrote:
>
> > Sorry that I didn't see this comment before the meeting Joel. I'll try
> to
> > clarify w
Hi,
I'm +1 for adding generic message headers, but I do share the concerns
previously aired on this thread and during the KIP meeting.
So let me propose a slimmer alternative that does not require any sort of
global header registry, does not affect broker performance or operations,
and adds as li
Hey,
I'm probably going to argue against bumping the SaslHandshake version, it
is redundant to the existing SaslHandshakeResponse.enabled_mechanisms field.
With SaslHandshake version bump:
* Client performs ApiVersionRequest
* If SaslHandshake >= v1: use SCRAM
* If SaslHandshake < v1: use
5 micro seconds to process
> a
> >> >> >>> message.
> >> >> >>>>>
> >> >> >>>>>
> >> >> >>>>>
> >> >> >>>>> I love strings as much as the nex
+1 (non-binding) - passes librdkafka regression tests
2017-04-18 17:38 GMT+02:00 Edoardo Comar :
> Thanks Gwen
> KAFKA-5075 is not included in the
> http://home.apache.org/~gwenshap/kafka-0.10.2.1-rc2/RELEASE_NOTES.html
>
> --
> Edoardo Comar
> IBM
I very much agree on what Joe is saying, let's use the version field as
intended
and be very strict with not removing nor altering existing behaviour
without bumping the version.
Old API versions could be deprecated (documentation only?) immediately and
removed completely in the next minor version
It is a bit hacky, with multiple requests in transit on the wire how do
you know which request wasn't supported if the returned correlationId is -1?
Requests are currently responded to in the order they were sent, so the
corrId
is not yet critical, but with will this always be true? One could imagi
Wouldn't it make sense to move away from these rich binary broker
descriptors ({ host, port, proto })
(which require protocol churning on change), and simply use URIs instead?
E.g.:
kafka://[:port]/ <-- cleantext proto on standard port 9092
kafkas://[:port] <-- SSL enveloped proto on s
+1 on librdkafka interop
Minor nitpick:
KAFKA-1781 (state required gradle version in README) is included in the
Release notes but is not actually fixed
2015-01-29 6:22 GMT+01:00 Jun Rao :
> This is the third candidate for release of Apache Kafka 0.8.2.0.
>
> Release Notes for the 0.8.2.0 rele
Hi Dongjin and good work on the KIP,
I understand that ZStandard is generally considered an improvement over
LZ4, but the
benchmark you provided on the KIP-110 wiki doesn't really reflect that, and
even
makes a note that they are comparable:
*> As you can see above, ZStandard shows outstanding per
Big +1 on VarInts.
CPUs are fast, memory is slow.
I agree with Jason that we'll want to continue verifying messages,
including their headers, so while I appreciate the idea of the opaque
header blob it won't be useful in practice.
/Magnus
2017-02-17 10:41 GMT-08:00 Jason Gustafson :
> Sorry, my
Hi all,
sorry for joining late in the game, the carribean got in the way.
My thoughts:
There is no way around the chicken&egg problem, so the sooner we can
add protocol versioning functionality the better and we'll add heuristics
in clients to
handle the migration period (e.g, what Dana has done
is
> > > > > > > > > in
> > > > > > > > > > > > >> 0.10.0.0.
> > > > > > > > > > > > >> >> > >
> > > > > > > > > > > >
OT will have
> >> different protocol than others (because we modify the protocol
> >> multiple times between releases)?
> >>
> >> Gwen
> >>
> >> On Thu, Mar 10, 2016 at 1:52 PM, Magnus Edenhill
> >> wrote:
> >> > Hi all,
>
I can update the KIP-35 wiki, WIP PR and start a Vote
> thread.
>
> On Fri, Mar 11, 2016 at 12:48 PM, Magnus Edenhill
> wrote:
>
> > I'm not sure supporting specific interim versions between releases are
> > really that big of a concern,
> > for a start th
Hey Jay,
as discussed earlier it is not safe to cache/relay a broker's version or
its supported API versions,
by the time the client connects the broker might have upgraded to another
version which effectively
makes this information useless in a cached form.
The complexity of querying for protoco
mixing cluster wide
> state about partitions with info about the answering broker.
>
> -Jay
>
> On Tue, Mar 15, 2016 at 4:25 PM, Magnus Edenhill
> wrote:
> > Hey Jay,
> >
> > as discussed earlier it is not safe to cache/relay a broker's version or
>
sically
> > > > > as we do with leadership information).
> > > > > 2. You check each connection
> > > > >
> > > > > I think combining metadata request and version check only makes
> sense
> > > > > in (1), right? If it i
+1 (nonbinding)
2016-03-18 0:09 GMT+01:00 Ashish Singh :
> As the KIP has been modified since we started this vote, the vote is
> restarted from now.
>
> The updated KIP is available at
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-35+-+Retrieving+protocol+version
> .
>
> On Mon, Mar
eed to be used. This is obviously more
> > tedious for more complex client-server interactions such as the new
> > consumer's rebalance process; but not so much for other features such as
> > KIP-32; or reading throttle time in produce/fetch responses that were
> > delay
As Jun says the SASL (and SSL) handshake is not done using the Kafka
protocol
and is performed before any Kafka protocol requests pass between client and
server.
It might make sense to move the SASL handshake from its custom protocol
format
into the Kafka protocol and make it use the proper Kafka
Empty arrays are already used in MetadataRequest to retrieve all topics in
the cluster,
ApiVersion request will have the same standard semantics.
2016-04-05 19:01 GMT+02:00 Ewen Cheslack-Postava :
> Also, just a thought but is empty list the sentinel we really want to
> indicate we want all API
> On Tue, Apr 5, 2016 at 10:23 AM, Magnus Edenhill
> wrote:
>
> > Empty arrays are already used in MetadataRequest to retrieve all topics
> in
> > the cluster,
> > ApiVersion request will have the same standard semantics.
> >
> >
> > 2016-04-05 19:01
this point.
Thanks,
Magnus
2016-04-05 19:40 GMT+02:00 Magnus Edenhill :
> Ashish, thanks, didnt know that.
>
> For ApiVersionRequest requesting no Apis to be returned doesnt make sense
> so the distinction isn't necessary,
> but I'm fine with adding Null to be more in
Engineers,
sorry to stir up some dust on this KIP but following yesterday's discussion
on the KIP call I'm a bit concerned about the way the SASL handshake
protocol is diverging even more from the main Kafka protocol.
The SASL handshake is already its own special protocol and the latest
addition o
s or request
versions in this special protocol?
I.e., what KIP-35 provides for the Kafka protocol.
Regards,
Magnus
>
> On Wed, Apr 6, 2016 at 12:47 PM, Magnus Edenhill
> wrote:
>
> > Engineers,
> >
> > sorry to stir up some dust on this KIP but following yesterday'
Hi Jun,
the "send empty response instead of disconnect on unsupported protocol"
was actually a sub-proposal and was never meant as an alternative to the
proper feature detection proposed by KIP-35.
I've added it back to the list of rejected alternatives on the wiki page,
thanks
for pointing this
2016-04-11 3:01 GMT+02:00 Jun Rao :
> Thinking about ApiVersionRequest a bit more. There are quite a few things
> special about it. In the ideal case, (1) its version should never change;
>
The only thing we know of the future is that we dont know anything, we can't
think of every possible future
is change since only two clients, Java and C,
currently supports
the existing SASL handshake so far.
>
> Thanks,
>
> Jun
>
> On Mon, Apr 11, 2016 at 12:20 AM, Magnus Edenhill
> wrote:
>
> > 2016-04-11 3:01 GMT+02:00 Jun Rao :
> >
> > > Thinking abo
Good point Rajini, I will clarify that.
Thanks,
Magnus
2016-04-22 12:35 GMT-07:00 Rajini Sivaram :
> +1 (non-binding)
>
> One minor comment:
>
> "11: The broker returns its full list of supported ApiKeys and versions
> regardless of current authentication state (e.g., before SASL
> authenticatio
Rajini,
the KIP wiki is a bit unclear on the protocol changes.
Could you document the proposed Kafka protocol requests&responses in the
standard format (as on "A guide to the Kafka protocol").
This information should also be added to that page when the KIP is accepted.
I think it would also be goo
+1 (non-binding)
2016-04-25 14:26 GMT-07:00 Dana Powers :
> Hi all,
>
> Initiating a vote thread because the KIP-57 proposal is specific to
> the 0.10 release.
>
> KIP-57 can be accessed here:
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-57+-+Interoperable+LZ4+Framing
> >.
>
> The
with the handshake
> request/response format. Have also added some more text to distinguish the
> actual authentication flow from the Kafka handshake/request flow.
>
> Thank you,
>
> Rajini
>
>
> On Mon, Apr 25, 2016 at 3:41 AM, Magnus Edenhill
> wrote:
>
> &g
Good work on this Dana.
I'll test it with librdkafka (which uses the official liblz4) and report
back.
2016-05-03 20:02 GMT+02:00 Dana Powers :
> Yes, great point. The intent of adding "naive" support for the
> remaining LZ4 header flags (contentsize and contentchecksum) is to
> avoid rejecting
I have verified this functionality on kafka trunk using librdkafka lz4
branch and it works as intended.
2016-05-07 18:07 GMT+02:00 Dana Powers :
> Vote Passed! I will update the wiki.
>
> -Dana
> On May 7, 2016 3:48 AM, "Ismael Juma" wrote:
>
> > Dana, a long time has passed since the vote sta
Hey,
could I have access to assign myself to issues in the Apache JIRA?
Username is edenhill.
Looking to provide some patches for KAFKA-3160.
Regards,
Magnus
Thanks!
2016-01-28 17:36 GMT+01:00 Guozhang Wang :
> Done.
>
> Guozhang
>
> On Thu, Jan 28, 2016 at 4:16 AM, Magnus Edenhill
> wrote:
>
> > Hey,
> >
> > could I have access to assign myself to issues in the Apache JIRA?
> > Username is edenhill.
Hi Abejide,
the ConsumerMetadataRequest was renamed to GroupCoordinatorRequest to
indicate its more generic use than just consumer groups.
See the protocol guide for more info:
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-GroupMembershipA
Hi,
since dots seem to be a problem on the metrics side, why not let the
metrics side handle it
by escaping troublesome characters? E.g. "foo.my\.topic.feh"
Let's not push the problem upstream.
Replacing "." with another set of allowed characters "__" seems like a bad
idea since it
is ambigious:
Good write-up Qin, the API looks promising.
I have one comment:
2016-09-03 5:20 GMT+02:00 Becket Qin :
> The currently offsetsForTimes() API obviously does not support querying
> multiple timestamps for the same partition. It doesn't seems a feature for
> ListOffsetRequest v0 either (sounds more
+1 (non-binding) passes librdkafka test suites
2016-10-19 15:55 GMT+02:00 Ismael Juma :
> +1 (non-binding).
>
> Verified source and Scala 2.11 binary artifacts, ran ./gradlew test with
> JDK 7u80, quick start on source artifact and Scala 2.11 binary artifacts.
>
> Thanks for managing the release!
Hi Michael,
With the version bumps for Produce and Fetch requests, do you really need
to bump MagicByte too?
Regards,
Magnus
2016-10-25 18:09 GMT+02:00 Michael Pearce :
> Hi All,
>
> I would like to discuss the following KIP proposal:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>
it would be flagged up in this discussion
> thread ☺
>
> Cheers
> Mike
>
> On 10/25/16, 5:36 PM, "Magnus Edenhill" wrote:
>
> Hi Michael,
>
> With the version bumps for Produce and Fetch requests, do you really
> need
> to bump MagicByte too
dont believe this is needed since it is already attributed through the
request's API version.
Producer:
* if a client sends ProduceRequest V4 then attributes.bit5 indicates a
tombstone
* if a clients sends ProduceRequest On Tue, Oct 25, 2016 at 10:17 AM, Magnus Edenhill
> wrote:
>
&
to 1 for this change.
> > But since we are not adding an actual field, we can do away with bumping
> up
> > the magic byte.
> >
> > If we really want to go the standard route of bumping up the magic byte
> for
> > any change to message format we should actually add a n
Great proposal, this feature is well overdue!
1)
>From an operator's perspective I don't think the kafka client
implementation name and version are sufficient,
I also believe the application name and version are of interest.
You could have all applications in your cluster run the same kafka client
Great KIP as always, Colin!
Some comments:
> If the flexible versions are not specified, it is assumed that all
versions are flexible.
This is ambiguous, if a protocol-generator is pointed to an older Kafka
protocol specification
it can't know if the lack of flexibleVersions field means they're
Hey Guozhang,
I'm late to the game, again, but if it is not too late I'd recommend that
instead
of having a single error code for all failed records, instead provide
per-record
error codes by having an array of [relative_offset, error_code], where
they're
both varint-encoded for space-efficiency.
. 2019 kl 08:49 skrev Magnus Edenhill :
> Hey Guozhang,
>
> I'm late to the game, again, but if it is not too late I'd recommend that
> instead
> of having a single error code for all failed records, instead provide
> per-record
> error codes by having an array
> I think we should force client software names and versions to follow a
regular expression and disconnect if they do not.
Disconnecting is not really a great error propagation method since it
leaves the client oblivious to what went wrong.
Instead suggest we return an ApiVersionResponse with an e
Den mån 23 sep. 2019 kl 14:42 skrev Colin McCabe :
> On Fri, Sep 20, 2019, at 18:05, Jun Rao wrote:
> > 101. We already use varInt in the message format. I assume that the
> > protocol uses the same varInt representation?
>
> It uses a slightly different varint representation. Basically, the
> di
Hi Tom,
I think it would be useful with some real world (or made up!) numbers on
how much relative/% space is saved for
the most error-dense protocol requests.
E.g., an OffsetCommitResponse with 10 topics and 100 failing partitions
would reduce the overall size by % bytes.
Thanks,
Magnus
Den ti
Hi David,
kudos on a very well designed KIP, this will make a ton of difference for
the maintainability of client implementations and operational aspects of
consumer groups!
Some comments:
1. Maybe I missed this, but where does max.poll.interval.ms fit into the
new protocol?
It seems like th
Great work on the KIP, David.
+1 (nonbinding)
Den fre 14 okt. 2022 kl 11:50 skrev Luke Chen :
> Hi David,
>
> I made a final pass and LGTM now.
> +1 from me.
>
> Luke
>
> On Wed, Oct 5, 2022 at 12:32 AM Guozhang Wang wrote:
>
> > Hello David,
> >
> > I've made my final pass on the doc and I thi
Hi, one minor comment on the latest update:
Den mån 24 okt. 2022 kl 16:26 skrev David Jacot :
> * Jason pointed out that the member id handling is a tad weird. The
> group coordinator generates the member id and then trusts the member
> when it rejoins the group. This also implies that the clien
Hi Travis and thanks for the KIP, two comments below:
Den fre 11 nov. 2022 kl 13:37 skrev David Jacot :
> 02: I am a bit concerned by clients that could misuse these information.
> For instance, one may be tempted to rely on the version to decide whether a
> feature is enabled or not. The api ve
Hi Colin,
aren't those counters (ever increasing), rather than gauges (fluctuating)?
You also mention CPU usage as a side note, you could use getrusage(2)'s
ru_utime (user) and ru_stime (sys)
to allow the broker to monitor its own CPU usage.
/Magnus
Den tis 10 dec. 2019 kl 19:33 skrev Colin McC
e proposal listed idempotence=true. This is more of a
> configuration
> > > than a metric. Are we including that as a metric? What other
> > configurations
> > > are we including? Should we separate the configurations from the
> metrics?
> > >
> > &
and outside the borders of the enterprise. Our clients like the performance
> and data safeties related to the kafka connection. The observability has
> been a problem...
>
> Jonathan Rivers
> jrivers...@gmail.com
>
>
>
>
> On Mon, Oct 18, 2021 at 11:56 PM Ryanne Dol
permitting,
and only secondarily as a Gauge average.
While we might not want to maintain a bunch of histograms for each
partition, since that could be
quite space consuming, this client.io.wait.time is a single metric per
client instance and can
thus afford a Histogram representation.
Thank
ull metric name
> >> including groups, tags, etc? We typically don't add the broker_id
> >> label for
> >> broker metrics. Also, brokers use Yammer metrics, which doesn't have
> >> type
> >> Sum.
> >>
> >> 29.
Hey Ismael,
> > The PushTelemetryRequest handler decompresses the payload before passing
> it
> > to the metrics plugin.
> > This was done to avoid having to expose a public decompression interface
> to
> > metrics plugin developers.
> >
>
> Are there cases where the metrics plugin developers wou
atic for large deployments. Still, I think it is worth
> > refactoring.
> >
> > IMHO, it is a fair trade-off.
> >
> > — Ricardo
> >
> > > On Jan 26, 2022, at 9:34 AM, Magnus Edenhill
> wrote:
> > >
> > > Hi all,
> > >
&g
Den ons 18 maj 2022 kl 19:57 skrev Jun Rao :
> Hi, Magnus,
>
Hi Jun
>
> Thanks for the updated KIP. Just a couple of more comments.
>
> 50. To troubleshoot a particular client issue, I imagine that the client
> needs to identify its client_instance_id. How does the client find this
> out? Do we
o suitable
standard metric a client can still
provide what special metrics it has.
Thanks,
Magnus
On Thu, May 19, 2022 at 10:39 AM Magnus Edenhill wrote:
>
> > Den ons 18 maj 2022 kl 19:57 skrev Jun Rao :
> >
> > > Hi, Magnus,
> > >
> >
> > Hi J
ubtle differences across clients seem to cause more confusion during
> > troubleshooting.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, May 31, 2022 at 5:02 AM Magnus Edenhill
> > wrote:
> >
> >> Den fre 20 maj 2022 kl 01:23 skrev Jun Rao :
&g
sh vs
> > dot)
> > > different from other existing metrics also seems a bit confusing. It
> > seems
> > > that the main benefit of having standard metric names across clients is
> > for
> > > better server side monitoring. Could we do the standardization in the
>
Hey Thomas,
I'm late to the game.
It looks like the "top level" ErrorCode moved from the top-level to the
Group array, which makes sense,
but it would be good if it was marked as // MOVED in the KIP and also a
note that top level errors that
are unrelated to the group will be returned as per-grou
Hey all,
I'm proposing KIP-714 to add remote Client metrics and observability.
This functionality will allow centralized monitoring and troubleshooting of
clients and their internals.
Please see
https://cwiki.apache.org/confluence/display/KAFKA/KIP-714%3A+Client+metrics+and+observability
Looking
Hey Colin,
good KIP!
Some comments:
1a. For operation, permission_type and resource_type: is there any reason
for having the any and unknown enums as negative values?
Since neither of these fields has an integer significance (unlike for
example offsets which use negative offsets for logical offs
Hi Rajini, great KIP!
This solution was proposed on the original KIP-43 thread but voted down, so
let's hope it does better this time :)
/Magnus
2017-05-04 13:37 GMT+02:00 Rajini Sivaram :
> Hi all,
>
> I have created a KIP to improve diagnostics for SASL authentication
> failures and reduce re
e, because then Confluent could provide drop-in hooks for any
> client
> and the end result of easy-panels would be the same.
>
> In summary,
>
> - Metrics are more of an organizational concern, not specifically a broker
> operator concern.
>
> - The proposal seems to hija
tor", so the opt-out appears to be the only motivation. Am I missing
> something?
>
> Ryanne
>
> On Wed, Jun 2, 2021 at 7:46 AM Magnus Edenhill wrote:
>
> > Hey all,
> >
> > I'm proposing KIP-714 to add remote Client metrics and observability.
> > This
Thanks for your feedback, Colin, see response below.
Den tors 17 juni 2021 kl 00:28 skrev Colin McCabe :
> On Sun, Jun 13, 2021, at 21:51, Travis Bischel wrote:
>
...
> > Another downside is that by dictating the important metrics, this KIP
> either
>
> has two choices: try to choose what is
Den tors 17 juni 2021 kl 00:52 skrev Colin McCabe :
> Hi Magnus,
>
> Thanks for the KIP. This is certainly something I've been wishing for for
> a while.
>
> Maybe we should emphasize more that the metrics that are being gathered
> here are Kafka metrics, not general application business logic met
ply "use a supported client". Even if this
> > > (potentially unlikely) scenario is true, then hooks would still be a
> great
> > > alternative, because then Confluent could provide drop-in hooks for
> any client
> > > and the end result of e
Hey Tom,
Den mån 21 juni 2021 kl 21:08 skrev Tom Bentley :
>
> 1. Did you consider using a `default ClientTelemetryReceiver
> clientReceiver() { return null; }` method on the existing MetricsReporter
> interface, avoiding the need for the ClientTelemetry trait?
>
I'll let Xavier answer this one
Hi Konstantine,
Some findings from running 3.0.0-RC1 with the librdkafka test suite:
* Compaction seems to take slightly longer to kick in when segment sizes
exceed their threshold. (Used to take less than 20 seconds, now takes
20..30 seconds.)
* CreateTopic seems to take slightly longer to pr
Thanks for your feedback Colin, see my updated proposal below.
Den tors 22 juli 2021 kl 03:17 skrev Colin McCabe :
> On Tue, Jun 29, 2021, at 07:22, Magnus Edenhill wrote:
> > Den tors 17 juni 2021 kl 00:52 skrev Colin McCabe :
> > > A few critiques:
> > >
> > &
Den mån 20 sep. 2021 kl 20:41 skrev Colin McCabe :
> On Tue, Sep 14, 2021, at 00:47, Magnus Edenhill wrote:
> > Thanks for your feedback Colin, see my updated proposal below.
> > ...
>
> Hi Magnus,
>
> Thanks for the update.
>
> >
> > Splitting up the A
Den tis 21 sep. 2021 kl 06:58 skrev Colin McCabe :
> On Mon, Sep 20, 2021, at 17:35, Feng Min wrote:
> > Thanks Magnus & Colin for the discussion.
> >
> > Based on KIP-714's stateless design, Client can pretty much use any
> > connection to any broker to send metrics. We are not associating
> conn
ouldn't
> find the vote thread. Is there one that I'm missing?
>
> Gwen
>
> On Wed, Sep 22, 2021 at 4:58 AM Magnus Edenhill
> wrote:
>
> > Den tis 21 sep. 2021 kl 06:58 skrev Colin McCabe :
> >
> > > On Mon, Sep 20, 2021, at 17:35
day or two.
Regards,
Magnus
Den mån 4 okt. 2021 kl 20:57 skrev Magnus Edenhill :
> Hi Gwen,
>
> I'm finishing up the KIP based on the last couple of discussion points in
> this thread
> and will call the Vote later this week.
>
> Best,
> Magnus
>
> De
Hi all,
I'd like to start a vote on KIP-714.
https://cwiki.apache.org/confluence/x/2xRRCg
Discussion thread:
https://www.mail-archive.com/dev@kafka.apache.org/msg119000.html
Thanks,
Magnus
plication instance id as an optional future nice-to-have
> that may be included as a metrics label if it has been set by the user", so
> I'm confused whether non-Kafka Streams clients should set an application_id
> or not.
>
I'll clarify this in the KIP, but basicall
number of partition/topics/etc the client is producing
to/consuming from.
I'll add some sizes to the KIP for some typical use-cases.
Thanks,
Magnus
> Thanks
>
> On Tue, Oct 19, 2021 at 5:06 PM Magnus Edenhill
> wrote:
> >
> > Den tis 19 okt. 2021 kl 13:22 skrev Tom Be
1 - 100 of 163 matches
Mail list logo