Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi lsmael I will update the KIP to replace “ org.apache.kafka.clients.tool” by “ org.apache.kafka.tools” due to following reasons. 1) it is more consistent with existent server/clients pluggable interface package 2) kafka tool jar is not expected to be imported to write code for pluggable interfaces. 3) no new module is introduced. The client module already has pluggable interface of tool, and thus putting them together can make users write other plugins with single (client) jar — Chia-Ping > Ismael Juma 於 2023年3月15日 上午3:06 寫道: > > Hi Chia, > > Regarding `org.apache.kafka.clients.tool`, a few comments: > > 1. Why is it in `clients`? We don't generally consider tools to be a client. > 2. Why is it `tool`? We have a package `org.apache.kafka.tools`, so it's a > bit odd that this one uses singular instead of plural. > 3. Also, should we follow the `storage-api` example and have a module for > all extensible interfaces used by tools? > > Ismael > >> On Tue, Mar 14, 2023 at 9:23 AM Chia-Ping Tsai wrote: >> >> >> Chris Egerton 於 2023年3月15日 上午12:04 寫道: >>> >>> Hi Chia-Ping, >>> >>> Thanks for the KIP. I find the interface definition really polished and >>> intuitive! One small question--I noticed the change of the package to >>> "org.apache.kafka.clients.tool". It doesn't look like there's any >> precedent >>> for using that package. We also use the "org.apache.kafka.common" package >>> for the "MessageFormatter" interface, which is in some ways the >> equivalent >>> pluggable interface for the console consumer. >> >> It seems to me those pluggable interfaces (MessageFormatter and >> RecordReader) should not be a part of “common” package. They are used by >> specify tools only. `Configurable`, by contrast, is good to be located at >> `common` package since it is used widely in our code base. >> >> >>> >>> Do we know if it's necessary to preserve the Checkstyle import >> limitations >>> (which I'm assuming are what motivated the shift in package name)? It >> seems >>> like it might be better to just relax that constraint in order to >> colocate >>> the pluggable interfaces for our console producer/consumer. >> >> I love checkstyle import, and that is one of reason the KIP isolates the >> new interface to a separate package. We have to add `allowed rule` one by >> one if those dedicated interfaces are using the `common` package. The >> constraint of new package can be relax to colocate the pluggable interfaces >> (used by tools), and the `relax` won’t impact other existent packages. >> >> >>> >>> Cheers, >>> >>> Chris >>> >>> On Tue, Mar 7, 2023 at 6:30 AM Chia-Ping Tsai >> wrote: >>> hi Mickael > ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > guess these can be removed now. Done! thanks for feedback > Mickael Maison 於 2023年3月7日 下午7:13 寫道: > > Hi Chia-Ping, > > The new API looks good. > I still see mentions to configure(InputStream inputStream, Map ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > guess these can be removed now. > > Thanks, > Mickael > > On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai wrote: >> >> Dear all, >> >> there are some changes for KIP-614 >> >> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 >> >> 1) the interface RecordReader extends Configurable. >> 2) the input stream is removed from RecordReader#configure method >> 3) RecordReader#readRecords accept InputStream as argument, and the returned type is changed from single ProducerRecord to Iterator >> >> Please take a look and then start to vote if you have free time. >> thanks. >> >> vote: >> https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr >> >>
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi juma > Ismael Juma 於 2023年3月15日 上午3:04 寫道: > > Hi Chia, > > Regarding `org.apache.kafka.clients.tool`, a few comments: > > 1. Why is it in `clients`? We don't generally consider tools to be a client. > 2. Why is it `tool`? We have a package `org.apache.kafka.tools`, so it's a > bit odd that this one uses singular instead of plural. `org.apache.kafka.tools` is a good package. However, it is used by tool module already. Not sure whether it is good idea to use same package in different module. Maybe it is fine since we don’t expect user has to import both tool jar and client jar for developing plugins. > 3. Also, should we follow the `storage-api` example and have a module for > all extensible interfaces used by tools? That seems good to me. However, it is a bit overkill if only few pluggable interfaces comes from tools. The rule I followed is server-side plugins (for example, AlterConfigPolicy, Authorizer, CreateTopicPolicy, and so on. In short, I considered `org.apache.kafka.tools` before, but it is used by tool module already. If different modules has same package does not store up trouble for the future, I prefer the package ``org.apache.kafka.tools`` — Chia-ping > > Ismael > > On Tue, Mar 14, 2023 at 9:23 AM Chia-Ping Tsai wrote: > >> >> >>> Chris Egerton 於 2023年3月15日 上午12:04 寫道: >>> >>> Hi Chia-Ping, >>> >>> Thanks for the KIP. I find the interface definition really polished and >>> intuitive! One small question--I noticed the change of the package to >>> "org.apache.kafka.clients.tool". It doesn't look like there's any >> precedent >>> for using that package. We also use the "org.apache.kafka.common" package >>> for the "MessageFormatter" interface, which is in some ways the >> equivalent >>> pluggable interface for the console consumer. >> >> It seems to me those pluggable interfaces (MessageFormatter and >> RecordReader) should not be a part of “common” package. They are used by >> specify tools only. `Configurable`, by contrast, is good to be located at >> `common` package since it is used widely in our code base. >> >> >>> >>> Do we know if it's necessary to preserve the Checkstyle import >> limitations >>> (which I'm assuming are what motivated the shift in package name)? It >> seems >>> like it might be better to just relax that constraint in order to >> colocate >>> the pluggable interfaces for our console producer/consumer. >> >> I love checkstyle import, and that is one of reason the KIP isolates the >> new interface to a separate package. We have to add `allowed rule` one by >> one if those dedicated interfaces are using the `common` package. The >> constraint of new package can be relax to colocate the pluggable interfaces >> (used by tools), and the `relax` won’t impact other existent packages. >> >> >>> >>> Cheers, >>> >>> Chris >>> >>> On Tue, Mar 7, 2023 at 6:30 AM Chia-Ping Tsai >> wrote: >>> hi Mickael > ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > guess these can be removed now. Done! thanks for feedback > Mickael Maison 於 2023年3月7日 下午7:13 寫道: > > Hi Chia-Ping, > > The new API looks good. > I still see mentions to configure(InputStream inputStream, Map ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > guess these can be removed now. > > Thanks, > Mickael > > On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai wrote: >> >> Dear all, >> >> there are some changes for KIP-614 >> >> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 >> >> 1) the interface RecordReader extends Configurable. >> 2) the input stream is removed from RecordReader#configure method >> 3) RecordReader#readRecords accept InputStream as argument, and the returned type is changed from single ProducerRecord to Iterator >> >> Please take a look and then start to vote if you have free time. >> thanks. >> >> vote: >> https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr >> >>
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia, Regarding `org.apache.kafka.clients.tool`, a few comments: 1. Why is it in `clients`? We don't generally consider tools to be a client. 2. Why is it `tool`? We have a package `org.apache.kafka.tools`, so it's a bit odd that this one uses singular instead of plural. 3. Also, should we follow the `storage-api` example and have a module for all extensible interfaces used by tools? Ismael On Tue, Mar 14, 2023 at 9:23 AM Chia-Ping Tsai wrote: > > > > Chris Egerton 於 2023年3月15日 上午12:04 寫道: > > > > Hi Chia-Ping, > > > > Thanks for the KIP. I find the interface definition really polished and > > intuitive! One small question--I noticed the change of the package to > > "org.apache.kafka.clients.tool". It doesn't look like there's any > precedent > > for using that package. We also use the "org.apache.kafka.common" package > > for the "MessageFormatter" interface, which is in some ways the > equivalent > > pluggable interface for the console consumer. > > It seems to me those pluggable interfaces (MessageFormatter and > RecordReader) should not be a part of “common” package. They are used by > specify tools only. `Configurable`, by contrast, is good to be located at > `common` package since it is used widely in our code base. > > > > > > Do we know if it's necessary to preserve the Checkstyle import > limitations > > (which I'm assuming are what motivated the shift in package name)? It > seems > > like it might be better to just relax that constraint in order to > colocate > > the pluggable interfaces for our console producer/consumer. > > I love checkstyle import, and that is one of reason the KIP isolates the > new interface to a separate package. We have to add `allowed rule` one by > one if those dedicated interfaces are using the `common` package. The > constraint of new package can be relax to colocate the pluggable interfaces > (used by tools), and the `relax` won’t impact other existent packages. > > > > > > Cheers, > > > > Chris > > > > On Tue, Mar 7, 2023 at 6:30 AM Chia-Ping Tsai > wrote: > > > >> hi Mickael > >> > >>> ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > >>> guess these can be removed now. > >> > >> Done! thanks for feedback > >> > >>> Mickael Maison 於 2023年3月7日 下午7:13 寫道: > >>> > >>> Hi Chia-Ping, > >>> > >>> The new API looks good. > >>> I still see mentions to configure(InputStream inputStream, Map >>> ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > >>> guess these can be removed now. > >>> > >>> Thanks, > >>> Mickael > >>> > >>> On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai > >> wrote: > > Dear all, > > there are some changes for KIP-614 > > > >> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > > 1) the interface RecordReader extends Configurable. > 2) the input stream is removed from RecordReader#configure method > 3) RecordReader#readRecords accept InputStream as argument, and the > >> returned type is changed from single ProducerRecord to > >> Iterator > > Please take a look and then start to vote if you have free time. > thanks. > > vote: > https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr > >> > >> > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Fair enough 👍 On Tue, Mar 14, 2023 at 12:23 PM Chia-Ping Tsai wrote: > > > > Chris Egerton 於 2023年3月15日 上午12:04 寫道: > > > > Hi Chia-Ping, > > > > Thanks for the KIP. I find the interface definition really polished and > > intuitive! One small question--I noticed the change of the package to > > "org.apache.kafka.clients.tool". It doesn't look like there's any > precedent > > for using that package. We also use the "org.apache.kafka.common" package > > for the "MessageFormatter" interface, which is in some ways the > equivalent > > pluggable interface for the console consumer. > > It seems to me those pluggable interfaces (MessageFormatter and > RecordReader) should not be a part of “common” package. They are used by > specify tools only. `Configurable`, by contrast, is good to be located at > `common` package since it is used widely in our code base. > > > > > > Do we know if it's necessary to preserve the Checkstyle import > limitations > > (which I'm assuming are what motivated the shift in package name)? It > seems > > like it might be better to just relax that constraint in order to > colocate > > the pluggable interfaces for our console producer/consumer. > > I love checkstyle import, and that is one of reason the KIP isolates the > new interface to a separate package. We have to add `allowed rule` one by > one if those dedicated interfaces are using the `common` package. The > constraint of new package can be relax to colocate the pluggable interfaces > (used by tools), and the `relax` won’t impact other existent packages. > > > > > > Cheers, > > > > Chris > > > > On Tue, Mar 7, 2023 at 6:30 AM Chia-Ping Tsai > wrote: > > > >> hi Mickael > >> > >>> ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > >>> guess these can be removed now. > >> > >> Done! thanks for feedback > >> > >>> Mickael Maison 於 2023年3月7日 下午7:13 寫道: > >>> > >>> Hi Chia-Ping, > >>> > >>> The new API looks good. > >>> I still see mentions to configure(InputStream inputStream, Map >>> ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > >>> guess these can be removed now. > >>> > >>> Thanks, > >>> Mickael > >>> > >>> On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai > >> wrote: > > Dear all, > > there are some changes for KIP-614 > > > >> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > > 1) the interface RecordReader extends Configurable. > 2) the input stream is removed from RecordReader#configure method > 3) RecordReader#readRecords accept InputStream as argument, and the > >> returned type is changed from single ProducerRecord to > >> Iterator > > Please take a look and then start to vote if you have free time. > thanks. > > vote: > https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr > >> > >> > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
> Chris Egerton 於 2023年3月15日 上午12:04 寫道: > > Hi Chia-Ping, > > Thanks for the KIP. I find the interface definition really polished and > intuitive! One small question--I noticed the change of the package to > "org.apache.kafka.clients.tool". It doesn't look like there's any precedent > for using that package. We also use the "org.apache.kafka.common" package > for the "MessageFormatter" interface, which is in some ways the equivalent > pluggable interface for the console consumer. It seems to me those pluggable interfaces (MessageFormatter and RecordReader) should not be a part of “common” package. They are used by specify tools only. `Configurable`, by contrast, is good to be located at `common` package since it is used widely in our code base. > > Do we know if it's necessary to preserve the Checkstyle import limitations > (which I'm assuming are what motivated the shift in package name)? It seems > like it might be better to just relax that constraint in order to colocate > the pluggable interfaces for our console producer/consumer. I love checkstyle import, and that is one of reason the KIP isolates the new interface to a separate package. We have to add `allowed rule` one by one if those dedicated interfaces are using the `common` package. The constraint of new package can be relax to colocate the pluggable interfaces (used by tools), and the `relax` won’t impact other existent packages. > > Cheers, > > Chris > > On Tue, Mar 7, 2023 at 6:30 AM Chia-Ping Tsai wrote: > >> hi Mickael >> >>> ?> configs) in the Compatibility, Deprecation, and Migration Plan, I >>> guess these can be removed now. >> >> Done! thanks for feedback >> >>> Mickael Maison 於 2023年3月7日 下午7:13 寫道: >>> >>> Hi Chia-Ping, >>> >>> The new API looks good. >>> I still see mentions to configure(InputStream inputStream, Map>> ?> configs) in the Compatibility, Deprecation, and Migration Plan, I >>> guess these can be removed now. >>> >>> Thanks, >>> Mickael >>> >>> On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai >> wrote: Dear all, there are some changes for KIP-614 >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 1) the interface RecordReader extends Configurable. 2) the input stream is removed from RecordReader#configure method 3) RecordReader#readRecords accept InputStream as argument, and the >> returned type is changed from single ProducerRecord to >> Iterator Please take a look and then start to vote if you have free time. thanks. vote: https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr >> >>
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Thanks for the KIP. I find the interface definition really polished and intuitive! One small question--I noticed the change of the package to "org.apache.kafka.clients.tool". It doesn't look like there's any precedent for using that package. We also use the "org.apache.kafka.common" package for the "MessageFormatter" interface, which is in some ways the equivalent pluggable interface for the console consumer. Do we know if it's necessary to preserve the Checkstyle import limitations (which I'm assuming are what motivated the shift in package name)? It seems like it might be better to just relax that constraint in order to colocate the pluggable interfaces for our console producer/consumer. Cheers, Chris On Tue, Mar 7, 2023 at 6:30 AM Chia-Ping Tsai wrote: > hi Mickael > > > ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > > guess these can be removed now. > > Done! thanks for feedback > > > Mickael Maison 於 2023年3月7日 下午7:13 寫道: > > > > Hi Chia-Ping, > > > > The new API looks good. > > I still see mentions to configure(InputStream inputStream, Map > ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > > guess these can be removed now. > > > > Thanks, > > Mickael > > > > On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai > wrote: > >> > >> Dear all, > >> > >> there are some changes for KIP-614 > >> > >> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > >> > >> 1) the interface RecordReader extends Configurable. > >> 2) the input stream is removed from RecordReader#configure method > >> 3) RecordReader#readRecords accept InputStream as argument, and the > returned type is changed from single ProducerRecord to > Iterator > >> > >> Please take a look and then start to vote if you have free time. thanks. > >> > >> vote: https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
hi Mickael > ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > guess these can be removed now. Done! thanks for feedback > Mickael Maison 於 2023年3月7日 下午7:13 寫道: > > Hi Chia-Ping, > > The new API looks good. > I still see mentions to configure(InputStream inputStream, Map ?> configs) in the Compatibility, Deprecation, and Migration Plan, I > guess these can be removed now. > > Thanks, > Mickael > > On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai wrote: >> >> Dear all, >> >> there are some changes for KIP-614 >> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 >> >> 1) the interface RecordReader extends Configurable. >> 2) the input stream is removed from RecordReader#configure method >> 3) RecordReader#readRecords accept InputStream as argument, and the returned >> type is changed from single ProducerRecord to Iterator >> >> Please take a look and then start to vote if you have free time. thanks. >> >> vote: https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, The new API looks good. I still see mentions to configure(InputStream inputStream, Map configs) in the Compatibility, Deprecation, and Migration Plan, I guess these can be removed now. Thanks, Mickael On Fri, Mar 3, 2023 at 2:37 PM Chia-Ping Tsai wrote: > > Dear all, > > there are some changes for KIP-614 > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > > 1) the interface RecordReader extends Configurable. > 2) the input stream is removed from RecordReader#configure method > 3) RecordReader#readRecords accept InputStream as argument, and the returned > type is changed from single ProducerRecord to Iterator > > Please take a look and then start to vote if you have free time. thanks. > > vote: https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Dear all, there are some changes for KIP-614 https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 1) the interface RecordReader extends Configurable. 2) the input stream is removed from RecordReader#configure method 3) RecordReader#readRecords accept InputStream as argument, and the returned type is changed from single ProducerRecord to Iterator Please take a look and then start to vote if you have free time. thanks. vote: https://lists.apache.org/thread/kjdtyfg5xytn60q0qvxhfopzmfp9tsxr
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi David > > - The new interface looks good to me. Note that the javadoc does not > reflect the new interface though. > - Could we precise how errors will be handled? For instance, say that the > iterator could translate the input stream to a record. Would calling the > next method on the iterator throw an exception? Thanks for kind reminder. I have updated the KIP. Please take a look. Thanks > David Jacot 於 2023年2月27日 下午8:26 寫道: > > Thanks for the update. > > - The new interface looks good to me. Note that the javadoc does not > reflect the new interface though. > - Could we precise how errors will be handled? For instance, say that the > iterator could translate the input stream to a record. Would calling the > next method on the iterator throw an exception? > > Thanks, > David > > On Sat, Feb 25, 2023 at 10:43 PM Chia-Ping Tsai wrote: > >> >> >> On 2023/02/25 08:26:28 David Jacot wrote: >>> Hi Chia-Ping, >>> >>> Thanks for the KIP. >>> >>> I find the configure method in the proposed interface a bit weird for a >> few >>> reasons. First, it has a default implementation which suggests that it is >>> optional but it is not because the InputStream is required. Second, it >> >> oh, my bad. I forgot to remove the default impl after adding the input >> stream to config method. >> >> >>> >>> Did we consider using two methods instead of one? We could have configure >>> coming from Configurable et setInputStream to set the InputStream. >> Another >>> option would be to have a method which takes the input stream and returns >>> an iterator to consume the records. >> >> I prefer to set input stream only once. Also, if Configurable interface is >> required for all plugins in kafka code base, the option.2 is suitable - we >> can change the returned type of `readRecords(InputStream)` from single >> record to an iterator of records. Thus, the new interface not only extends >> Configurable but also take input stream only once. >> >> >>> >>> Cheers, >>> David >>> >>> Le mer. 22 févr. 2023 à 11:53, Chia-Ping Tsai a >>> écrit : >>> On 2023/02/22 10:01:29 Alexandre Dupriez wrote: > Hi Chia-Ping, > > Thanks for your answer. Apologies I should have been clearer in the > previous message. What I meant is, is there a plan to use the SPI >> more > broadly inside the Kafka codebase? no, there is no plan to reuse the SPI. > The question arises because the interface exposes a close() method > which is never invoked by the ConsoleProducer. Hence, although we >> need > to keep this method to maintain compatibility of the SPI with its > current implementations yep, you are right. the close() method is never executed by the ConsoleProducer. The ConsolerConsumer has similar issue ( https://github.com/apache/kafka/pull/8978). I will fix it. > we should perhaps clarify that this method is > not used/deprecated, unless it is intended to be used in the future. I prefer to keep the close() since the new interface is similar to Deserializer. The close() method can be used to notify/release >> something when the console is going to be down. >>> >>
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Thanks for the update. - The new interface looks good to me. Note that the javadoc does not reflect the new interface though. - Could we precise how errors will be handled? For instance, say that the iterator could translate the input stream to a record. Would calling the next method on the iterator throw an exception? Thanks, David On Sat, Feb 25, 2023 at 10:43 PM Chia-Ping Tsai wrote: > > > On 2023/02/25 08:26:28 David Jacot wrote: > > Hi Chia-Ping, > > > > Thanks for the KIP. > > > > I find the configure method in the proposed interface a bit weird for a > few > > reasons. First, it has a default implementation which suggests that it is > > optional but it is not because the InputStream is required. Second, it > > oh, my bad. I forgot to remove the default impl after adding the input > stream to config method. > > > > > > Did we consider using two methods instead of one? We could have configure > > coming from Configurable et setInputStream to set the InputStream. > Another > > option would be to have a method which takes the input stream and returns > > an iterator to consume the records. > > I prefer to set input stream only once. Also, if Configurable interface is > required for all plugins in kafka code base, the option.2 is suitable - we > can change the returned type of `readRecords(InputStream)` from single > record to an iterator of records. Thus, the new interface not only extends > Configurable but also take input stream only once. > > > > > > Cheers, > > David > > > > Le mer. 22 févr. 2023 à 11:53, Chia-Ping Tsai a > > écrit : > > > > > > > > > > > On 2023/02/22 10:01:29 Alexandre Dupriez wrote: > > > > Hi Chia-Ping, > > > > > > > > Thanks for your answer. Apologies I should have been clearer in the > > > > previous message. What I meant is, is there a plan to use the SPI > more > > > > broadly inside the Kafka codebase? > > > > > > no, there is no plan to reuse the SPI. > > > > > > > The question arises because the interface exposes a close() method > > > > which is never invoked by the ConsoleProducer. Hence, although we > need > > > > to keep this method to maintain compatibility of the SPI with its > > > > current implementations > > > > > > yep, you are right. the close() method is never executed by the > > > ConsoleProducer. The ConsolerConsumer has similar issue ( > > > https://github.com/apache/kafka/pull/8978). I will fix it. > > > > > > > we should perhaps clarify that this method is > > > > not used/deprecated, unless it is intended to be used in the future. > > > > > > I prefer to keep the close() since the new interface is similar to > > > Deserializer. The close() method can be used to notify/release > something > > > when the console is going to be down. > > > > > > > > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
On 2023/02/25 08:26:28 David Jacot wrote: > Hi Chia-Ping, > > Thanks for the KIP. > > I find the configure method in the proposed interface a bit weird for a few > reasons. First, it has a default implementation which suggests that it is > optional but it is not because the InputStream is required. Second, it oh, my bad. I forgot to remove the default impl after adding the input stream to config method. > > Did we consider using two methods instead of one? We could have configure > coming from Configurable et setInputStream to set the InputStream. Another > option would be to have a method which takes the input stream and returns > an iterator to consume the records. I prefer to set input stream only once. Also, if Configurable interface is required for all plugins in kafka code base, the option.2 is suitable - we can change the returned type of `readRecords(InputStream)` from single record to an iterator of records. Thus, the new interface not only extends Configurable but also take input stream only once. > > Cheers, > David > > Le mer. 22 févr. 2023 à 11:53, Chia-Ping Tsai a > écrit : > > > > > > > On 2023/02/22 10:01:29 Alexandre Dupriez wrote: > > > Hi Chia-Ping, > > > > > > Thanks for your answer. Apologies I should have been clearer in the > > > previous message. What I meant is, is there a plan to use the SPI more > > > broadly inside the Kafka codebase? > > > > no, there is no plan to reuse the SPI. > > > > > The question arises because the interface exposes a close() method > > > which is never invoked by the ConsoleProducer. Hence, although we need > > > to keep this method to maintain compatibility of the SPI with its > > > current implementations > > > > yep, you are right. the close() method is never executed by the > > ConsoleProducer. The ConsolerConsumer has similar issue ( > > https://github.com/apache/kafka/pull/8978). I will fix it. > > > > > we should perhaps clarify that this method is > > > not used/deprecated, unless it is intended to be used in the future. > > > > I prefer to keep the close() since the new interface is similar to > > Deserializer. The close() method can be used to notify/release something > > when the console is going to be down. > > > > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Thanks for the KIP. I find the configure method in the proposed interface a bit weird for a few reasons. First, it has a default implementation which suggests that it is optional but it is not because the InputStream is required. Second, it diverges from the Configurable interface. Third, it is strange to pass an InputStream to a configure method. Did we consider using two methods instead of one? We could have configure coming from Configurable et setInputStream to set the InputStream. Another option would be to have a method which takes the input stream and returns an iterator to consume the records. Cheers, David Le mer. 22 févr. 2023 à 11:53, Chia-Ping Tsai a écrit : > > > On 2023/02/22 10:01:29 Alexandre Dupriez wrote: > > Hi Chia-Ping, > > > > Thanks for your answer. Apologies I should have been clearer in the > > previous message. What I meant is, is there a plan to use the SPI more > > broadly inside the Kafka codebase? > > no, there is no plan to reuse the SPI. > > > The question arises because the interface exposes a close() method > > which is never invoked by the ConsoleProducer. Hence, although we need > > to keep this method to maintain compatibility of the SPI with its > > current implementations > > yep, you are right. the close() method is never executed by the > ConsoleProducer. The ConsolerConsumer has similar issue ( > https://github.com/apache/kafka/pull/8978). I will fix it. > > > we should perhaps clarify that this method is > > not used/deprecated, unless it is intended to be used in the future. > > I prefer to keep the close() since the new interface is similar to > Deserializer. The close() method can be used to notify/release something > when the console is going to be down. > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
On 2023/02/22 10:01:29 Alexandre Dupriez wrote: > Hi Chia-Ping, > > Thanks for your answer. Apologies I should have been clearer in the > previous message. What I meant is, is there a plan to use the SPI more > broadly inside the Kafka codebase? no, there is no plan to reuse the SPI. > The question arises because the interface exposes a close() method > which is never invoked by the ConsoleProducer. Hence, although we need > to keep this method to maintain compatibility of the SPI with its > current implementations yep, you are right. the close() method is never executed by the ConsoleProducer. The ConsolerConsumer has similar issue (https://github.com/apache/kafka/pull/8978). I will fix it. > we should perhaps clarify that this method is > not used/deprecated, unless it is intended to be used in the future. I prefer to keep the close() since the new interface is similar to Deserializer. The close() method can be used to notify/release something when the console is going to be down.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Thanks for your answer. Apologies I should have been clearer in the previous message. What I meant is, is there a plan to use the SPI more broadly inside the Kafka codebase? The question arises because the interface exposes a close() method which is never invoked by the ConsoleProducer. Hence, although we need to keep this method to maintain compatibility of the SPI with its current implementations, we should perhaps clarify that this method is not used/deprecated, unless it is intended to be used in the future. Thanks, Alexandre Le mer. 22 févr. 2023 à 09:27, Chia-Ping Tsai a écrit : > > > A minor comment. The SPI is currently used exclusively for the > > ConsoleProducer. However, it exposes high-level methods which hint at > > it being a generic component. What is the actual scope of the SPI > > inside the Kafka codebase? Is it planned to be re-used in other tools? > > Or is this interface used (not implemented) outside of the > > ConsoleProducer? > > It is used by ConsoleProducer only. The interface is a kind of public APIs, > and IIRC the public APIs must be written by Java. That is why we need to move > it out of core module (simplify core module also). > > There are many other interfaces which allow users to "enhance" kafka. For > example, partitioner, assignor, authorizer, and so on. Most of them are used > exclusively for specify component, and I guess not all interfaces are widely > used (implemented). Maybe we can file a thread to cleanup the "unused" > interfaces.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
> A minor comment. The SPI is currently used exclusively for the > ConsoleProducer. However, it exposes high-level methods which hint at > it being a generic component. What is the actual scope of the SPI > inside the Kafka codebase? Is it planned to be re-used in other tools? > Or is this interface used (not implemented) outside of the > ConsoleProducer? It is used by ConsoleProducer only. The interface is a kind of public APIs, and IIRC the public APIs must be written by Java. That is why we need to move it out of core module (simplify core module also). There are many other interfaces which allow users to "enhance" kafka. For example, partitioner, assignor, authorizer, and so on. Most of them are used exclusively for specify component, and I guess not all interfaces are widely used (implemented). Maybe we can file a thread to cleanup the "unused" interfaces.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Thank you for the KIP and apologies for missing it earlier. A minor comment. The SPI is currently used exclusively for the ConsoleProducer. However, it exposes high-level methods which hint at it being a generic component. What is the actual scope of the SPI inside the Kafka codebase? Is it planned to be re-used in other tools? Or is this interface used (not implemented) outside of the ConsoleProducer? Thanks, Alexandre Le sam. 18 févr. 2023 à 19:02, Chia-Ping Tsai a écrit : > > > > On 2023/02/18 08:44:05 Tom Bentley wrote: > > Hi Chia-Ping, > > > > To be honest the stateful version, setting an input stream once using the > > `readFrom(InputStream)` method and then repeatedly asking for the next > > record using a parameterless `readRecord()`, seems a bit more natural to me > > than `readRecord(InputStream inputStream)` being called repeatedly with (I > > assume) the same input stream. I think the contract is simpler to describe > > and understand. > > I prefer readRecord() also. It is a trade-off between having `Configurable` > interface and having a parameterless readRecord(). If the `Configurable` is > not required, I'd like to revert to readRecord(). WDYT? > > > > > It's worth thinking about how implementers might have to read bytes from > > the stream to discover the end of one record and the start of the next. > > Unless we've guaranteed that the input stream supports mark and reset then > > they have to buffer the initial bytes of the next record that they've just > > read from the stream so that they can use them when called next time. So I > > think RecordReaders are (in general) inherently stateful and therefore it > > seems harmless for them to also have the input stream itself as some of > > that state. > > you are right. As the input stream is keyboard input, it would be hard to > expect the number of bytes for one record. >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
On 2023/02/18 08:44:05 Tom Bentley wrote: > Hi Chia-Ping, > > To be honest the stateful version, setting an input stream once using the > `readFrom(InputStream)` method and then repeatedly asking for the next > record using a parameterless `readRecord()`, seems a bit more natural to me > than `readRecord(InputStream inputStream)` being called repeatedly with (I > assume) the same input stream. I think the contract is simpler to describe > and understand. I prefer readRecord() also. It is a trade-off between having `Configurable` interface and having a parameterless readRecord(). If the `Configurable` is not required, I'd like to revert to readRecord(). WDYT? > > It's worth thinking about how implementers might have to read bytes from > the stream to discover the end of one record and the start of the next. > Unless we've guaranteed that the input stream supports mark and reset then > they have to buffer the initial bytes of the next record that they've just > read from the stream so that they can use them when called next time. So I > think RecordReaders are (in general) inherently stateful and therefore it > seems harmless for them to also have the input stream itself as some of > that state. you are right. As the input stream is keyboard input, it would be hard to expect the number of bytes for one record.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Dear all, I noticed a issue after starting the vote :( The package "org.apache.kafka.common" disallows to import "org.apache.kafka.clients.producer.ProducerRecord". Hence, "org.apache.kafka.common.RecordReader" can't reference "org.apache.kafka.clients.producer.ProducerRecord" directly. It seems to me the tool-related interface should be able to access all public classes from "common", "producer", "consumer", and "admin". Accordingly, I'd like to move the package of "RecordReader" from "org.apache.kafka.common" to ""org.apache.kafka.clients.tool". Please take a look at above change. thanks, chia-ping
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, To be honest the stateful version, setting an input stream once using the `readFrom(InputStream)` method and then repeatedly asking for the next record using a parameterless `readRecord()`, seems a bit more natural to me than `readRecord(InputStream inputStream)` being called repeatedly with (I assume) the same input stream. I think the contract is simpler to describe and understand. It's worth thinking about how implementers might have to read bytes from the stream to discover the end of one record and the start of the next. Unless we've guaranteed that the input stream supports mark and reset then they have to buffer the initial bytes of the next record that they've just read from the stream so that they can use them when called next time. So I think RecordReaders are (in general) inherently stateful and therefore it seems harmless for them to also have the input stream itself as some of that state. Cheers, Tom On Sat, 18 Feb 2023 at 08:25, Chia-Ping Tsai wrote: > > > On 2023/02/17 06:47:18 Luke Chen wrote: > > Hi Chia-Ping, > > > > Thanks for the KIP! > > > > Overall LGTM, just one minor comment: > > Could we log warning messages to users when using deprecated > MessageReader? > > Sure. I will address it when implementing the KIP. > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
On 2023/02/17 06:47:18 Luke Chen wrote: > Hi Chia-Ping, > > Thanks for the KIP! > > Overall LGTM, just one minor comment: > Could we log warning messages to users when using deprecated MessageReader? Sure. I will address it when implementing the KIP.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Thanks for the KIP! Overall LGTM, just one minor comment: Could we log warning messages to users when using deprecated MessageReader? Thank you. Luke On Fri, Feb 17, 2023 at 2:04 PM Chia-Ping Tsai wrote: > hi Bentley > > I have updated the KIP to make RecordReader extend Configurable. PTAL >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
hi Bentley I have updated the KIP to make RecordReader extend Configurable. PTAL
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
> 3. On the matter of configure(): While it doesn't add any functionality, it > would be more consistent with other plugins if this interface extended > Configurable, and the InputStream was then passed via some other method > (`readFrom(InputStream)` perhaps). If nothing else it would make it harder > to overlook this interface when making changes which apply to all plugins. > To be honest, I'm not entirely convinced myself, but I thought we should at > least consider it and add it to the rejected alternatives if we decide > against it. Make sense. How about moving the `InputStream` from `configure` to `readRecord`? By that change, the `RecordReader` can be a subinterface of `Configurable`.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
On 2023/02/16 11:26:04 Federico Valeri wrote: > Hi Chia-Ping, thanks for updating the KIP. > > I would only add that we plan to remove the old trait in the next > major release. I think it's better to be explicit about this. sure. I have added this description to doc about deprecation.
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Thanks for the KIP. I have just a few minor comments: 1. The Javadoc for the new interface says "an `InputStream` received via `init`" but the interface doesn't have an init() method. I guess you meant configure()? 2. The Javadoc should mention the need for implementations to have a public nullary constructor. 3. On the matter of configure(): While it doesn't add any functionality, it would be more consistent with other plugins if this interface extended Configurable, and the InputStream was then passed via some other method (`readFrom(InputStream)` perhaps). If nothing else it would make it harder to overlook this interface when making changes which apply to all plugins. To be honest, I'm not entirely convinced myself, but I thought we should at least consider it and add it to the rejected alternatives if we decide against it. Thanks again, Tom On Thu, 16 Feb 2023 at 11:36, Federico Valeri wrote: > Hi Chia-Ping, thanks for updating the KIP. > > I would only add that we plan to remove the old trait in the next > major release. I think it's better to be explicit about this. > > On Thu, Feb 16, 2023 at 11:34 AM Chia-Ping Tsai > wrote: > > > > Dear all, > > > > I have updated the KIP to address comments. PTAL > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, thanks for updating the KIP. I would only add that we plan to remove the old trait in the next major release. I think it's better to be explicit about this. On Thu, Feb 16, 2023 at 11:34 AM Chia-Ping Tsai wrote: > > Dear all, > > I have updated the KIP to address comments. PTAL > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Dear all, I have updated the KIP to address comments. PTAL https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
On 2023/02/15 14:29:06 Ismael Juma wrote: > Hi. > > Is MessageReader the right name? The client apis tend to use the word > `Record` instead. In fact, `readMessage` returns a `ProducerRecord`. Make sense. “Record” is the better naming. I will update KIP to use “Record” instead of “message” > > Also, I wonder if we it would be useful for the interface to support the > usage of Serializers. That is, the interface could return ProducerRecord > as long as a serializer was also configured. I am less sure about this one, > but worth giving it some thought and if we think it's a bad idea, we should > add it to the "Rejected Alternatives" section. It seems to me the serializer should be a part of implementation of MessageReader. If the custom serializer is supported, users have to define two class (reader and serializer) to convert bytes (from input stream) to “correct records”. I will add it to dejected alternatives > > Ismael > > On Wed, Feb 15, 2023 at 3:07 AM Mickael Maison > wrote: > > > Hi Chia-Ping, > > > > Sorry nobody replied to your thread earlier! > > I stumbled on this while looking at > > https://issues.apache.org/jira/browse/KAFKA-14525 (migrating CLI tools > > out of core). Your proposal still makes sense and would help migrating > > the ConsoleProducer tool. > > > > Can we specify when we expect to delete kafka.common.MessageReader? I > > guess in Kafka 4.0? > > Apart from that, the KIP looks good and considering how simple it is, > > you should be able to open a vote thread. > > > > Thanks, > > Mickael > > > > On Tue, Jul 7, 2020 at 5:55 PM Chia-Ping Tsai wrote: > > > > > > hi all, > > > > > > I would like to start the discussion for KIP-641. > > > > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > > > > > > Many thanks, > > > > > > Chia-Ping > > >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi. Is MessageReader the right name? The client apis tend to use the word `Record` instead. In fact, `readMessage` returns a `ProducerRecord`. Also, I wonder if we it would be useful for the interface to support the usage of Serializers. That is, the interface could return ProducerRecord as long as a serializer was also configured. I am less sure about this one, but worth giving it some thought and if we think it's a bad idea, we should add it to the "Rejected Alternatives" section. Ismael On Wed, Feb 15, 2023 at 3:07 AM Mickael Maison wrote: > Hi Chia-Ping, > > Sorry nobody replied to your thread earlier! > I stumbled on this while looking at > https://issues.apache.org/jira/browse/KAFKA-14525 (migrating CLI tools > out of core). Your proposal still makes sense and would help migrating > the ConsoleProducer tool. > > Can we specify when we expect to delete kafka.common.MessageReader? I > guess in Kafka 4.0? > Apart from that, the KIP looks good and considering how simple it is, > you should be able to open a vote thread. > > Thanks, > Mickael > > On Tue, Jul 7, 2020 at 5:55 PM Chia-Ping Tsai wrote: > > > > hi all, > > > > I would like to start the discussion for KIP-641. > > > > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > > > > Many thanks, > > > > Chia-Ping >
Re: [DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
Hi Chia-Ping, Sorry nobody replied to your thread earlier! I stumbled on this while looking at https://issues.apache.org/jira/browse/KAFKA-14525 (migrating CLI tools out of core). Your proposal still makes sense and would help migrating the ConsoleProducer tool. Can we specify when we expect to delete kafka.common.MessageReader? I guess in Kafka 4.0? Apart from that, the KIP looks good and considering how simple it is, you should be able to open a vote thread. Thanks, Mickael On Tue, Jul 7, 2020 at 5:55 PM Chia-Ping Tsai wrote: > > hi all, > > I would like to start the discussion for KIP-641. > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 > > Many thanks, > > Chia-Ping
[DISCUSS] KIP-641 An new java interface to replace 'kafka.common.MessageReader'
hi all, I would like to start the discussion for KIP-641. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866569 Many thanks, Chia-Ping