;
> >> >>
> >> >>>
> >> >>>>
> >> >>>>>
> >> >>>>>
> >> >>>>> default RLMM implementation?
> >> >>>>>
t; > remoteLogSegmentId) throws IOException;
> > > > > 1003.2 There seems to be some inconsistencies in the methods below.
> > I am
> > > > > not sure why one takes RemoteLogSegmentMetadata and the other
> > > > > takes Re
Jun Rao wrote:
> Hi, Ying,
>
> Thanks for the update. It's good to see the progress on this. Please let us
> know when you are done updating the KIP wiki.
>
> Jun
>
> On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng wrote:
>
> > Hi Jun,
> >
> > Sati
;>>>>
> > > > > >>>>>>> Br,
> > > > > >>>>>>> Ivan
> > > > > >>>>>>>
> > > > > >>>>>>> [1]
> > > > > >>>>>>>
> > > > > >>>&
Please ignore my previous email
I didn't know Apache requires all the discussions to be "open"
On Tue, Nov 19, 2019, 5:40 PM Ying Zheng wrote:
> Hi Jun,
>
> Thank you very much for your feedback!
>
> Can we schedule a meeting in your Palo Alto office in Dece
at KIP-392 (allow consumers to fetch from closest replica) is
> implemented, we need to discuss how reading from a follower replica is
> supported with tier storage.
> 43.4 We need to discuss how JBOD is supported with tier storage.
>
> Thanks,
>
> Jun
>
> On Fri, Nov 8, 2019
On Fri, Nov 15, 2019 at 11:04 AM Jun Rao wrote:
> Hi, Satish and Harsha,
>
> The following is a more detailed high level feedback for the KIP. Overall,
> the KIP seems useful. The challenge is how to design it such that it’s
> general enough to support different ways of implementing this feature
>
>
>
> Thanks, I missed that point. However, there's still a point at which the
> consumer fetches start getting served from remote storage (even if that
> point isn't as soon as the local log retention time/size). This represents
> a kind of performance cliff edge and what I'm really interested i
On Wed, Nov 6, 2019 at 6:28 PM Tom Bentley wrote:
> Hi Ying,
>
> Because only inactive segments can be shipped to remote storage, to be able
> > to ship log data as soon
> > as possible, we will roll log segment very fast (e.g. every half hour).
> >
>
> So that means a consumer which gets behind
On Wed, Nov 6, 2019 at 4:33 PM Ying Zheng wrote:
> 21. I am not sure that I understood the need for RemoteLogIndexEntry and
> its relationship with RemoteLogSegmentInfo. It seems
> that RemoteLogIndexEntry are offset index entries pointing to record
> batches inside a segment. That
21. I am not sure that I understood the need for RemoteLogIndexEntry and
its relationship with RemoteLogSegmentInfo. It seems
that RemoteLogIndexEntry are offset index entries pointing to record
batches inside a segment. That seems to be the same as the .index file?
We do not assume the how the da
On Thu, Oct 24, 2019 at 7:13 AM Eno Thereska wrote:
> Going back to initial thread with general questions on KIP. I think
> aspects of the user experience still need clarification:
>
> - if a user has a mix of compacted and non-compacted topics it will be
> hard to reason about storage needs over
>
>
>
>
- Why wait until local segments expire before offloading them to cold
> storage? Why not stream to HDFS/S3 on an ongoing basis? I'd think this
> would reduce bursty behavior from periodic uploads.
>
>
I think you misunderstood the KIP. We do plan to ship the segment files to
remote storage
an old go (sarama) client library which is known to have
> some
> > > > serious bugs
> > > > (3) force kafka 1.x clients with the ability to roll back if there's
> an
> > > > issue (unlike a message format upgrade)
> > > >
> > > &
;
>
> Am I missing something? Are there other examples that are not related to
> message conversion?
>
> Ismael
>
> On Thu, Apr 11, 2019 at 11:53 PM Ying Zheng
> wrote:
>
> > Hi here,
> >
> > Please vote for
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-433%3A+Block+old+clients+on+brokers
> >
> > Thank you!
> >
>
client to know what happened? Is there a way
> for the admin to know how many clients are rejected?
>
> As a nit, the "migration plan" part of the KIP still mentions the
> authorizer.
>
> Gwen
>
> On Thu, Apr 11, 2019 at 2:53 PM Ying Zheng wrote:
>
> > Hi he
Hi here,
Please vote for
https://cwiki.apache.org/confluence/display/KAFKA/KIP-433%3A+Block+old+clients+on+brokers
Thank you!
ys directly is reasonable here.
>
> One thing that's probably worth calling out is that if we make the name
> part of the configuration, we can't rename APIs in the future. That's
> probably OK as long as it's documented.
>
> best,
> Colin
>
> On Thu
On Wed, Mar 27, 2019 at 5:32 PM Colin McCabe wrote:
> Thanks, Ying Zheng. Looks good overall.
>
> One question is, should the version be specified as a Kafka version rather
> than as a RPC API version? I don't think most users are aware of RPC
> versions, but something
; > > > a configuration is clearly the better way to go here. Perhaps it
> can
> > > > > be a KIP-226 dynamic configuration to make this easier to deploy?
> > > > >
> > > > > cheers,
> > > > > Colin
> > >
If you guys don't like the extension of authorizer interface, I will just
propose a single broker dynamic configuration: client.min.api.version, to
keep things simple.
What do you think?
On Mon, Feb 25, 2019 at 2:23 PM Ying Zheng wrote:
> @Viktor Somogyi-Vass, @Harsha, It seems the
h 1.
>
> Thanks,
> Harsha
>
> On Sat, Feb 23, 2019, at 6:21 AM, Ismael Juma wrote:
> > Thanks for the KIP. Have we considered the existing topic config that
> makes
> > it possible to disallow down conversions? That's the biggest downside in
> > allowing older clients.
> >
> > Ismael
> >
> > On Fri, Feb 22, 2019, 2:11 PM Ying Zheng wrote:
> >
> > >
> > >
> >
>
7;t be done dynamically - configs can be dynamic and the
> admin can avoid a restart).
>
> Would be interested to hear what others think.
>
> Gwen
>
> On Fri, Feb 22, 2019 at 2:11 PM Ying Zheng wrote:
> >
> >
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>
Ying Zheng created KAFKA-7975:
-
Summary: Provide client API version to authorizer
Key: KAFKA-7975
URL: https://issues.apache.org/jira/browse/KAFKA-7975
Project: Kafka
Issue Type: Improvement
Ying Zheng created KAFKA-7142:
-
Summary: Rebalancing large consumer group can block the
coordinator broker for several seconds
Key: KAFKA-7142
URL: https://issues.apache.org/jira/browse/KAFKA-7142
Ying Zheng created KAFKA-6432:
-
Summary: Lookup indices may cause unnecessary page fault
Key: KAFKA-6432
URL: https://issues.apache.org/jira/browse/KAFKA-6432
Project: Kafka
Issue Type
Ying Zheng created KAFKA-6431:
-
Summary: Lock contention in Purgatory
Key: KAFKA-6431
URL: https://issues.apache.org/jira/browse/KAFKA-6431
Project: Kafka
Issue Type: Improvement
Ying Zheng created KAFKA-6430:
-
Summary: Improve Kafka GZip compression performance
Key: KAFKA-6430
URL: https://issues.apache.org/jira/browse/KAFKA-6430
Project: Kafka
Issue Type: Improvement
29 matches
Mail list logo