[jira] [Created] (KAFKA-15010) KRaft Controller doesn't reconcile with Zookeeper metadata upon becoming new controller while in dual write mode.

2023-05-19 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-15010:
-

 Summary: KRaft Controller doesn't reconcile with Zookeeper 
metadata upon becoming new controller while in dual write mode.
 Key: KAFKA-15010
 URL: https://issues.apache.org/jira/browse/KAFKA-15010
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Reporter: Akhilesh Chaganti


When a KRaft controller fails over, the existing migration driver (in dual 
write mode) can fail in between Zookeeper writes and may leave Zookeeper with 
incomplete and inconsistent data. So when a new controller becomes active (and 
by extension new migration driver becomes active), this first thing we should 
do is load the in-memory snapshot and use it to write metadata to Zookeeper to 
have a steady state. We currently do not do this and it may leave Zookeeper in 
inconsistent state.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15009) ClientQuotas and ACLs are not correctly synchronized while handling snapshot during migration (dual write)

2023-05-19 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-15009:
-

 Summary: ClientQuotas and ACLs are not correctly synchronized 
while handling snapshot during migration (dual write)
 Key: KAFKA-15009
 URL: https://issues.apache.org/jira/browse/KAFKA-15009
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.5.0
Reporter: Akhilesh Chaganti
Assignee: Akhilesh Chaganti






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] Apache Kafka 3.5.0 release

2023-05-19 Thread Akhilesh Chaganti
Hi Mickael,

I raised the blockers for AK 3.5 and raised PRs for them. All of them are
in the Zk -> KRaft migration path and critical for the Zk -> KRaft
migration to be successful.
KAFKA-15003  -- Topic
metadata is not correctly synced with Zookeeper while handling snapshot
during migration (dual write). --
KAFKA-15004  -- Config
changes are not correctly synced with Zookeeper whale handling snapshot
during migration (dual write). --
KAFKA-15007  --
MigrationPropagator may have wrong IBP while sending UMR, LISR requests to
the Zk Brokers during migration. --


Thanks
Akhilesh


On Fri, May 19, 2023 at 9:28 AM Matthias J. Sax  wrote:

> Mickael,
>
> we included a bug-fix into 3.5, and just discovered a critical bug in
> the fix itself, that would introduce a regression into 3.5.
>
> We have already a PR to fix-forward:
> https://github.com/apache/kafka/pull/13734
>
> As we don't have an RC yet, I would like to cherry-pick this back to 3.5.
>
>
> -Matthias
>
> On 5/10/23 1:47 PM, Sophie Blee-Goldman wrote:
> > Thanks Mickael -- the fix has been merged to 3.5 now
> >
> > On Wed, May 10, 2023 at 1:12 AM Mickael Maison  >
> > wrote:
> >
> >> Hi Sophie,
> >>
> >> Yes that's fine, thanks for letting me know!
> >>
> >> Mickael
> >>
> >> On Tue, May 9, 2023 at 10:54 PM Sophie Blee-Goldman
> >>  wrote:
> >>>
> >>> Hey Mickael, I noticed a bug in the new versioned key-value byte store
> >>> where it's delegating to the wrong API
> >>> (copy/paste error I assume). I extracted this into its own PR which I
> >> think
> >>> should be included in the 3.5 release.
> >>>
> >>> The tests are still running, but it's just a one-liner so I'll merge it
> >>> when they're done, and cherrypick to 3.5 if
> >>> that's ok with you. See https://github.com/apache/kafka/pull/13695
> >>>
> >>> Thanks for running the release!
> >>>
> >>> On Tue, May 9, 2023 at 1:28 PM Randall Hauch  wrote:
> >>>
>  Thanks, Mickael.
> 
>  I've cherry-picked that commit to the `3.5` branch (
>  https://issues.apache.org/jira/browse/KAFKA-14974).
> 
>  Best regards,
>  Randall
> 
>  On Tue, May 9, 2023 at 2:13 PM Mickael Maison <
> >> mickael.mai...@gmail.com>
>  wrote:
> 
> > Hi Randall/Luke,
> >
> > Yes you can go ahead and merge these into 3.5. I've not started
> >> making
> > a release yet because:
> > - I found a regression today in MirrorMaker:
> > https://issues.apache.org/jira/browse/KAFKA-14980
> > - The 3.5 branch builder job in Jenkins has been disabled:
> > https://issues.apache.org/jira/browse/INFRA-24577
> >
> > Thanks,
> > Mickael
> >
> > On Tue, May 9, 2023 at 8:40 PM Luke Chen  wrote:
> >>
> >> Hi Mickael,
> >>
> >> Since we haven't had the CR created yet, I'm thinking we should
>  backport
> >> this doc improvement to v3.5.0 to make the doc complete.
> >> https://github.com/apache/kafka/pull/13660
> >>
> >> What do you think?
> >>
> >> Luke
> >>
> >> On Sat, May 6, 2023 at 11:31 PM David Arthur 
> >> wrote:
> >>
> >>> I resolved these three:
> >>> * KAFKA-14840 is merged to trunk and 3.5. I removed the 3.4.1 fix
> > version
> >>> * KAFKA-14805 is merged to trunk and 3.5
> >>> * KAFKA-14918 is merged to trunk and 3.5
> >>>
> >>> KAFKA-14692 (docs issue) is still a not done
> >>>
> >>> Looks like KAFKA-14084 is now resolved as well (it's in trunk and
>  3.5).
> >>>
> >>> I'll try to find out about KAFKA-14698, I think it's likely a
>  WONTFIX.
> >>>
> >>> -David
> >>>
> >>> On Fri, May 5, 2023 at 10:43 AM Mickael Maison <
> > mickael.mai...@gmail.com>
> >>> wrote:
> >>>
>  Hi David,
> 
>  Thanks for the update!
>  You still own 4 other tickets targeting 3.5: KAFKA-14840,
> > KAFKA-14805,
>  KAFKA-14918, KAFKA-14692. Should I move all of them to the next
>  release?
>  Also KAFKA-14698 and KAFKA-14084 are somewhat related to the
>  migration. Should I move them too?
> 
>  Thanks,
>  Mickael
> 
>  On Fri, May 5, 2023 at 4:27 PM David Arthur
>   wrote:
> >
> > Hey Mickael, my two ZK migration fixes are in 3.5 now.
> >
> > Cheers,
> > David
> >
> > On Fri, May 5, 2023 at 9:37 AM Mickael Maison <
> >>> mickael.mai...@gmail.com>
> > wrote:
> >
> >> Hi Divij,
> >>
> >> Some dependencies (ZooKeeper, Snappy, Swagger, zstd, etc)
> >> have
> > been
> >> updated since 3.4.
> >> Regarding your PR, I would have been in favor of bringing
> >> this
> > to 3.5
> >> a couple of weeks ago, but we're now a week past code
> >> freeze
>  

[jira] [Created] (KAFKA-15007) MV is not set correctly in the MetadataPropagator in migration.

2023-05-18 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-15007:
-

 Summary: MV is not set correctly in the MetadataPropagator in 
migration.
 Key: KAFKA-15007
 URL: https://issues.apache.org/jira/browse/KAFKA-15007
 Project: Kafka
  Issue Type: Bug
Reporter: Akhilesh Chaganti


MV changes are not set in propagator unless we're in DUAL_WRITE mode. But we do 
this, we'll skip any known MV changes. The propagator should always know the 
correct MV so that it sends correct UMR and LISR during DUAL_WRITE.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15004) Topic config changes are not synced during zk to kraft migration (dual-write)

2023-05-17 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-15004:
-

 Summary: Topic config changes are not synced during zk to kraft 
migration (dual-write)
 Key: KAFKA-15004
 URL: https://issues.apache.org/jira/browse/KAFKA-15004
 Project: Kafka
  Issue Type: Bug
Reporter: Akhilesh Chaganti






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-15003) TopicIdReplicaAssignment is not updated in migration (dual-write) when partitions are changed for topic

2023-05-17 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-15003:
-

 Summary: TopicIdReplicaAssignment is not updated in migration 
(dual-write) when partitions are changed for topic
 Key: KAFKA-15003
 URL: https://issues.apache.org/jira/browse/KAFKA-15003
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.5.0
Reporter: Akhilesh Chaganti
 Fix For: 3.5.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14801) Encoded sensitive configs are not decoded before migration

2023-03-13 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14801:
-

 Summary: Encoded sensitive configs are not decoded before migration
 Key: KAFKA-14801
 URL: https://issues.apache.org/jira/browse/KAFKA-14801
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Reporter: Akhilesh Chaganti






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14637) Upgrade to 3.4 from old versions (< 0.10) are failing due to incompatible meta.properties check

2023-01-19 Thread Akhilesh Chaganti (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Akhilesh Chaganti resolved KAFKA-14637.
---
Fix Version/s: 3.4.0
   Resolution: Fixed

> Upgrade to 3.4 from old versions (< 0.10) are failing due to incompatible 
> meta.properties check
> ---
>
> Key: KAFKA-14637
> URL: https://issues.apache.org/jira/browse/KAFKA-14637
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 3.4.0
>    Reporter: Akhilesh Chaganti
>    Assignee: Akhilesh Chaganti
>Priority: Blocker
> Fix For: 3.4.0
>
>
> 3.4 has a check in broker startup to ensure cluster.id is provided in 
> `metadata.properties`. This is not always the case if the previous version of 
> Kafka is < 0.10.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14637) Upgrade to 3.4 from old versions (< 0.10) are failing due to incompatible meta.properties check

2023-01-18 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14637:
-

 Summary: Upgrade to 3.4 from old versions (< 0.10) are failing due 
to incompatible meta.properties check
 Key: KAFKA-14637
 URL: https://issues.apache.org/jira/browse/KAFKA-14637
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.4.0
Reporter: Akhilesh Chaganti


3.4 has a check in broker startup to ensure cluster.id is provided in 
`metadata.properties`. This is not always the case if the previous version of 
Kafka is < 0.10.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14493) Zk to KRaft migration state machine in KRaft controller

2022-12-14 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14493:
-

 Summary: Zk to KRaft migration state machine in KRaft controller
 Key: KAFKA-14493
 URL: https://issues.apache.org/jira/browse/KAFKA-14493
 Project: Kafka
  Issue Type: Sub-task
Reporter: Akhilesh Chaganti
Assignee: Akhilesh Chaganti






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14458) RPC Handler to ZkBrokers from KRaft Controller

2022-12-09 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14458:
-

 Summary: RPC Handler to ZkBrokers from KRaft Controller
 Key: KAFKA-14458
 URL: https://issues.apache.org/jira/browse/KAFKA-14458
 Project: Kafka
  Issue Type: Sub-task
Reporter: Akhilesh Chaganti
Assignee: Akhilesh Chaganti






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14446) API forwarding support in ZkBrokers

2022-12-06 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14446:
-

 Summary: API forwarding support in ZkBrokers
 Key: KAFKA-14446
 URL: https://issues.apache.org/jira/browse/KAFKA-14446
 Project: Kafka
  Issue Type: Sub-task
Reporter: Akhilesh Chaganti
Assignee: Akhilesh Chaganti


To support migration, zkBrokers should be able to forward API requests to the 
Controller, whether it is zkController or kraftController. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


Re: [DISCUSS] KIP-866 ZooKeeper to KRaft Migration

2022-11-01 Thread Akhilesh Chaganti
Hi David,


Thanks for the KIP. I have some questions/suggestions.


1) I see two new metrics:
kafka.controller:type=KafkaController,name=MetadataType and
kafka.controller:type=KafkaController,name=MigrationState. Won't the second
metric already cover the cases of the first metric? Also, instead of
MigrationFinalized, we could directly say the state is KRaftMode. So we can
use the same value for default KRaft clusters.


2) ZkMigrationReady in ApiVersionsResponse from KRaft Controller. By
default, we plan to start the Controller quorum in "
*kafka.metadata.migration.enable*" config set to true. Then do we need this
additional information again to make sure The controllers are ready for
migration? What would happen if the Controller assumes it is ready for
migration from 3.4 by default if it doesn't see both MigrationMetadata
records?


3) I see that we do not impose order on rolling the brokers with migration
flags and provisioning the controller quorum. Along with the KRaft
controller emitting migration state metrics, it may be better to emit the
broker count for the brokers not ready for migration yet. This will give us
more insight into any roll issues.


4) Once the KRaft controller is in migration mode, we should also
prevent/handle ZkBrokerRegistrations that don't enable migration mode.


Thanks
Akhilesh


On Tue, Nov 1, 2022 at 10:49 AM Jun Rao  wrote:

> Hi, David,
>
> Thanks for the reply.
>
> 20/21. Regarding the new ZkMigrationReady field in ApiVersionsResponse, it
> seems that this is a bit intrusive since it exposes unneeded info to the
> clients. Another option is to add that field as part of the Fetch request.
> We can choose to only set that field in the very first Fetch request from a
> Quorum follower.
>
> 40. For kafka.controller:type=KafkaController,name=MigrationState, what is
> the value for a brand new KRaft cluster?
>
> Jun
>
> On Mon, Oct 31, 2022 at 2:35 PM David Arthur
>  wrote:
>
> > 30. I think we can keep the single ControllerId field in those requests
> > since they are only used for fencing (as far as I know). Internally, the
> > broker components that handle those requests will compare the
> ControllerId
> > with that of MetadataCache (which is updated via UMR).
> >
> > The reason we need the separate KRaftControllerId in the UpdateMetadata
> > code path so that we can have different connection behavior for a KRaft
> > controller vs ZK controller.
> >
> > 31. It seems reasonable to keep the MigrationRecord in the snapshot. I
> was
> > thinking the same thing in terms of understanding the loss for a
> > migration-after-finalization. However, once a snapshot has been taken
> that
> > includes the final MigrationRecord, we can't easily see which records
> came
> > after it.
> >
> > 32. You're correct, we can just use the modify time from the Stat. The
> > other two fields are primarily informational and are there for operators
> > who want to inspect the state of the migration. They aren't required for
> > correctness
> >
> > 33. Yes that's right. I detail that in "Controller Leadership" section
> >
> > 34. Right, I'll fix that.
> >
> > Thanks,
> > David
> >
> > On Mon, Oct 31, 2022 at 2:55 PM Jun Rao 
> wrote:
> >
> > > Hi, David,
> > >
> > > Thanks for the updated KIP. A few more comments.
> > >
> > > 30. LeaderAndIsrRequest/StopReplicaRequest both have a controllerId
> > field.
> > > Should we add a KRaftControllerId field like UpdateMetadata?
> > >
> > > 31. "If a migration has been finalized, but the KRaft quroum comes up
> > with
> > > kafka.metadata.migration.enable, we must not re-enter the migration
> mode.
> > > In this case, while replaying the log, the controller can see the
> second
> > > MigrationRecord and know that the migration is finalized and should not
> > be
> > > resumed. " Hmm, do we want to keep the MigrationRecord in the snapshot
> > and
> > > the metadata log forever after migration is finalized? If not, we can't
> > > know for sure whether a migration has happened or not. Also, it might
> be
> > > useful to support switching back to ZK mode after the migration is
> > > finalized, with the understanding of potential metadata loss. In that
> > case,
> > > we could just trim all metadata log and recopy the ZK metadata back.
> > >
> > > 32. The /migration node in ZK: Do we need last_update_time_ms since ZK
> > > Stats already has an MTime? Also, how do we plan to use the
> > > kraft_controller_id and kraft_controller_epoch fields?
> > >
> > > 33. Controller migration: We will force a write to the "/controller"
> and
> > > "/controller_epoch" ZNodes before copying ZK data, right?
> > >
> > > 34. "Operator can remove the persistent "/controller" and
> > > "/controller_epoch" nodes allowing for ZK controller election to take
> > > place". I guess the operator only needs to remove the /controller path?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Mon, Oct 31, 2022 at 7:17 AM David Arthur
> > >  wrote:
> > >
> > > > Happy Monday, everyone! I've updated 

[jira] [Created] (KAFKA-14291) KRaft: ApiVersionsResponse doesn't have finalizedFeatures and finalizedFeatureEpoch in KRaft mode

2022-10-11 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14291:
-

 Summary: KRaft: ApiVersionsResponse doesn't have finalizedFeatures 
and finalizedFeatureEpoch in KRaft mode
 Key: KAFKA-14291
 URL: https://issues.apache.org/jira/browse/KAFKA-14291
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Reporter: Akhilesh Chaganti
 Fix For: 3.4.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14214) StandardAuthorizer may transiently process ACLs out of write order

2022-09-09 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14214:
-

 Summary: StandardAuthorizer may transiently process ACLs out of 
write order
 Key: KAFKA-14214
 URL: https://issues.apache.org/jira/browse/KAFKA-14214
 Project: Kafka
  Issue Type: Bug
Affects Versions: 3.3
Reporter: Akhilesh Chaganti


The issue with StandardAuthorizer#authorize is, that it looks up 
aclsByResources (which is of type ConcurrentSkipListMap)twice for every 
authorize call and uses Iterator with weak consistency guarantees on top of 
aclsByResources. This can cause the authorize function call to process the 
concurrent writes out of order.
*Issue 1:*
When StandardAuthorizer calls into a simple authorize function, we check the 
ACLs for literal/prefix matches for the resource and then make one more call to 
check the ACLs for matching wildcard entries. Between the two (checkSection) 
calls, let’s assume we add a DENY for resource literal and add an ALLOW ALL 
wildcard. The first call to check literal/prefix rules will SKIP DENY ACL since 
the writes are not yet processed and the second call would find ALLOW wildcard 
entry which results in ALLOW authorization for the resource when it is actually 
DENY.

*Issue: 2*

For authorization, StandardAuthorizer depends on an iterator that iterates 
through the ordered set of ACLs. The iterator has weak consistency guarantees. 
So when writes for two ACLs occur, one of the ACLs might be still visible to 
the iterator while the other is not. 

Let’s say below two ACLS are getting added in the following order to the set.
Acl1 = StandardAcl(TOPIC, foobar, LITERAL, DENY, READ, user1)
Acl2 = StandardAcl(TOPIC, foo, PREFIX, ALLOW, READ, user1)
Depending on the position of the iterator on the ordered set during the write 
call, the iterator might just see Acl2 which prompts it to ALLOW the topic to 
be READ even though the DENY rule was written before.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-14170) KRaft Controller: Possible NPE when we remove topics with any offline partitions in the cluster

2022-08-30 Thread Akhilesh Chaganti (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-14170?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Akhilesh Chaganti resolved KAFKA-14170.
---
Resolution: Fixed

> KRaft Controller: Possible NPE when we remove topics with any offline 
> partitions in the cluster
> ---
>
> Key: KAFKA-14170
> URL: https://issues.apache.org/jira/browse/KAFKA-14170
> Project: Kafka
>  Issue Type: Bug
>  Components: kraft
>Affects Versions: 3.3
>    Reporter: Akhilesh Chaganti
>    Assignee: Akhilesh Chaganti
>Priority: Blocker
> Fix For: 3.3
>
>
> When we remove a topic, it goes through the following function in KRaft 
> Controller replay method for RemoveTopicRecord:
> {code:java}
> void removeTopicEntryForBroker(Uuid topicId, int brokerId) {
> Map topicMap = isrMembers.get(brokerId);
> if (topicMap != null) {
> if (brokerId == NO_LEADER) {
> offlinePartitionCount.set(offlinePartitionCount.get() - 
> topicMap.get(topicId).length);
> }
> topicMap.remove(topicId);
> }
> } {code}
> If the broker has any offline partitions but doesn't have offline partitions 
> for the topic we're deleting, the above code will run into NPE because we 
> directly access the `topicMap.get(topicId).length`



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (KAFKA-14170) KRaft Controller: Possible NPE when we remove topics with any offline partitions in the cluster

2022-08-17 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-14170:
-

 Summary: KRaft Controller: Possible NPE when we remove topics with 
any offline partitions in the cluster
 Key: KAFKA-14170
 URL: https://issues.apache.org/jira/browse/KAFKA-14170
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Affects Versions: 3.3
Reporter: Akhilesh Chaganti
Assignee: Akhilesh Chaganti
 Fix For: 3.3


When we remove a topic, it goes through the following function in KRaft 
Controller replay method for RemoveTopicRecord:
{code:java}
void removeTopicEntryForBroker(Uuid topicId, int brokerId) {
Map topicMap = isrMembers.get(brokerId);
if (topicMap != null) {
if (brokerId == NO_LEADER) {
offlinePartitionCount.set(offlinePartitionCount.get() - 
topicMap.get(topicId).length);
}
topicMap.remove(topicId);
}
} {code}

If the broker has any offline partitions but doesn't have offline partitions 
for the topic we're deleting, the above code will run into NPE because we 
directly access the `topicMap.get(topicId).length`



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Resolved] (KAFKA-13861) validateOnly request field does not work for CreatePartition requests in Kraft mode.

2022-05-04 Thread Akhilesh Chaganti (Jira)


 [ 
https://issues.apache.org/jira/browse/KAFKA-13861?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Akhilesh Chaganti resolved KAFKA-13861.
---
Resolution: Fixed

> validateOnly request field does not work for CreatePartition requests in 
> Kraft mode.
> 
>
> Key: KAFKA-13861
> URL: https://issues.apache.org/jira/browse/KAFKA-13861
> Project: Kafka
>  Issue Type: Bug
>    Reporter: Akhilesh Chaganti
>    Assignee: Akhilesh Chaganti
>Priority: Major
>
> `ControllerApis` ignores the validateOnly field and the `QuorumController` 
> does not have any logic to handle the `validateOnly` requests for 
> `CreatePartitions.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[jira] [Created] (KAFKA-13861) validateOnly request field does not work for CreatePartition requests in Kraft mode.

2022-04-28 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-13861:
-

 Summary: validateOnly request field does not work for 
CreatePartition requests in Kraft mode.
 Key: KAFKA-13861
 URL: https://issues.apache.org/jira/browse/KAFKA-13861
 Project: Kafka
  Issue Type: Bug
Reporter: Akhilesh Chaganti
Assignee: Akhilesh Chaganti


`ControllerApis` ignores the validateOnly field and the `QuorumController` does 
not have any logic to handle the `validateOnly` requests for `CreatePartitions.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


Re: Need permission to be contributor on Apache Kafka project

2022-03-16 Thread Akhilesh Chaganti
My username is akhileshchg.

Thanks
Akhilesh


On Wed, Mar 16, 2022 at 2:46 PM Bill Bejeck  wrote:

> Hi Akhilesh,
>
> Can you provide your Jira username? We can get you setup after that.
>
> Thanks,
> Bill
>
> On Wed, Mar 16, 2022 at 4:38 PM Akhilesh Chaganti 
> wrote:
>
> > Hi,
> >
> > I raised a bug (https://issues.apache.org/jira/browse/KAFKA-13749) for
> > Apache Kafka. I want to submit a PR for this by becoming a contributor.
> Can
> > anyone help with getting the approval to assign the bug to me?
> >
> >
> > Thanks
> > Akhilesh
> >
>


Need permission to be contributor on Apache Kafka project

2022-03-16 Thread Akhilesh Chaganti
Hi,

I raised a bug (https://issues.apache.org/jira/browse/KAFKA-13749) for
Apache Kafka. I want to submit a PR for this by becoming a contributor. Can
anyone help with getting the approval to assign the bug to me?


Thanks
Akhilesh


[jira] [Created] (KAFKA-13749) TopicConfigs and ErrorCode are not set in createTopics response in KRaft

2022-03-16 Thread Akhilesh Chaganti (Jira)
Akhilesh Chaganti created KAFKA-13749:
-

 Summary: TopicConfigs and ErrorCode are not set in createTopics 
response in KRaft
 Key: KAFKA-13749
 URL: https://issues.apache.org/jira/browse/KAFKA-13749
 Project: Kafka
  Issue Type: Bug
  Components: kraft
Reporter: Akhilesh Chaganti


Once the createTopics request is process in KRaft, the `CreatableTopicResult` 
is not set with the appropriate topic configs and error and this breaks KIP-525



--
This message was sent by Atlassian Jira
(v8.20.1#820001)