Re: [DISCUSS] KIP-317: Transparent Data Encryption

2018-06-18 Thread Stephane Maarek
Hi Sonke Very much needed feature and discussion. FYI the image links seem broken. My 2 cents (if I understood correctly): you say "This process will be implemented after Serializer and Interceptors are done with the message right before it is added to the batch to be sent, in order to ensure tha

Re: [DISCUSS] - KIP-314: KTable to GlobalKTable Bi-directional Join

2018-06-18 Thread Matthias J. Sax
Adam, thanks a lot for the KIP. I agree that this would be a valuable feature to add. It's a very complex one though. You correctly pointed out, that the GlobalKTable (or global stores in general) cannot be the "driver" atm and are passively updated only. This is by design. Are you familiar with t

Build failed in Jenkins: kafka-trunk-jdk10 #232

2018-06-18 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-7067; Include new connector configs in system test assertion -- [...truncated 1.57 MB...] kafka.log.BrokerCompressionTest > testBrokerSideCompre

[jira] [Resolved] (KAFKA-7023) Kafka Streams RocksDB bulk loading config may not be honored with customized RocksDBConfigSetter

2018-06-18 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-7023. -- Resolution: Fixed Fix Version/s: 2.0.0 > Kafka Streams RocksDB bulk loading config may n

Re: Error in Kafka Stream

2018-06-18 Thread Guozhang Wang
Interesting, it indeed seem like a lurking issue in Kafka Streams. Which Kafka version are you using? Guozhang On Mon, Jun 18, 2018 at 12:32 AM, Amandeep Singh wrote: > Hi Guozhang, > > The file system is XFS and the folder is not a temp folder. The issue goes > away when I restart the strea

Jenkins build is back to normal : kafka-trunk-jdk8 #2749

2018-06-18 Thread Apache Jenkins Server
See

[DISCUSS] KIP-317: Transparent Data Encryption

2018-06-18 Thread Sönke Liebau
Hi everybody, I've created a draft version of KIP-317 which describes the addition of transparent data encryption functionality to Kafka. Please consider this as a basis for discussion - I am aware that this is not at a level of detail sufficient for implementation, but I wanted to get some feedb

[jira] [Resolved] (KAFKA-7067) ConnectRestApiTest fails assertion

2018-06-18 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7067?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7067. Resolution: Fixed > ConnectRestApiTest fails assertion > --

Re: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Guozhang Wang
Hi Matthias, Yes, we are effectively assigning the the whole space of Strings minus current preserved ones as header keys; honestly I think in practice users wanting to use `_something_` would be very rare, but I admit it may still be possible in theory. I think Luis' point about "header=" is tha

[jira] [Created] (KAFKA-7072) Kafka Streams may drop rocksb window segments before they expire

2018-06-18 Thread John Roesler (JIRA)
John Roesler created KAFKA-7072: --- Summary: Kafka Streams may drop rocksb window segments before they expire Key: KAFKA-7072 URL: https://issues.apache.org/jira/browse/KAFKA-7072 Project: Kafka

Re: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Matthias J. Sax
Luis, I meant to update the "Rejected Alternative" sections, what you have done already. Thx. Originally, I also had the idea about a second config, but thought it might be easier to just change the allowed values to be `offset`, `timestamp`, `header=`. (We try to keep the number of configs small

RE: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Luís Cabral
I’m ok with that... Ted / Matthias? From: Guozhang Wang Sent: 18 June 2018 22:49 To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-280: Enhanced log compaction How about make the preserved values to be "_offset_" and "_timestamp_" then? Currently in the KIP they are reserved as "offset" and "tim

[jira] [Resolved] (KAFKA-7071) specify number of partitions when using repartition logic

2018-06-18 Thread Boyang Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Boyang Chen resolved KAFKA-7071. Resolution: Duplicate Dup with 6037 > specify number of partitions when using repartition logic >

Re: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Guozhang Wang
How about make the preserved values to be "_offset_" and "_timestamp_" then? Currently in the KIP they are reserved as "offset" and "timestamp". Guozhang On Mon, Jun 18, 2018 at 1:40 PM, Luís Cabral wrote: > Hi Guozhang, > > Yes, that is what I meant (separate configs). > Though I would still

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-18 Thread Ted Yu
+1 On Mon, Jun 18, 2018 at 1:04 PM, Lucas Wang wrote: > Hi All, > > I've addressed a couple of comments in the discussion thread for KIP-291, > and > got no objections after making the changes. Therefore I would like to start > the voting thread. > > KIP: > https://cwiki.apache.org/confluence/di

RE: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Luís Cabral
Hi Guozhang, Yes, that is what I meant (separate configs). Though I would still prefer to keep it as it is, as its a much simpler and cleaner approach – I’m not so sure that a potential client would really be so inconvenienced for having to use “_offset” or “_timestamp_” as a header Cheers, Luí

[DISCUSS] - KIP-314: KTable to GlobalKTable Bi-directional Join

2018-06-18 Thread Adam Bellemare
Hi All I created KIP-314 and I would like to initiate a discussion on it. https://cwiki.apache.org/confluence/display/KAFKA/KIP-314%3A+KTable+to+GlobalKTable+Bi-directional+Join The primary goal of this KIP is to improve the way that Kafka can deal with relational data at scale. This KIP would a

[VOTE] KIP-291: Have separate queues for control requests and data requests

2018-06-18 Thread Lucas Wang
Hi All, I've addressed a couple of comments in the discussion thread for KIP-291, and got no objections after making the changes. Therefore I would like to start the voting thread. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-291%3A+Have+separate+queues+for+control+requests+and+data

[jira] [Resolved] (KAFKA-5237) SimpleConsumerShell logs terminating message to stdout instead of stderr

2018-06-18 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-5237. -- Resolution: Auto Closed Old consumer related tools are deprecated and  will be removed in KAFKA-2983.

[VOTE] KIP-316: Command-line overrides for ConnectDistributed worker properties

2018-06-18 Thread Kevin Lafferty
Hi all, I got a couple notes of interest on the discussion thread and no objections, so I'd like to kick off a vote. This is a very small change. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-316%3A+Command-line+overrides+for+ConnectDistributed+worker+properties Jira: https://issues

[jira] [Resolved] (KAFKA-4870) A question about broker down , the server is doing partition master election,the client producer may send msg fail . How the producer deal with the situation ??

2018-06-18 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4870?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4870. -- Resolution: Information Provided If the produce request fails, the producer automatically retry based o

Re: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Guozhang Wang
Hello Luís, I agree that having an expression evaluation as a config value is not the best approach; if there are better ideas to allow users to specify the header key which happen to be the same as the preserved config values "offset" and "timestamp" (although the likelihood may be small, as Ted

[jira] [Created] (KAFKA-7071) specify number of partitions when using repartition logic

2018-06-18 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-7071: -- Summary: specify number of partitions when using repartition logic Key: KAFKA-7071 URL: https://issues.apache.org/jira/browse/KAFKA-7071 Project: Kafka Issue Typ

[jira] [Resolved] (KAFKA-3791) Broken tools -- need better way to get offsets and other info

2018-06-18 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3791. -- Resolution: Fixed kafka-consumer-offset-checker.sh tool has been removed. Use kafka-consumer-groups.sh

[jira] [Resolved] (KAFKA-4061) Apache Kafka failover is not working

2018-06-18 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4061. -- Resolution: Cannot Reproduce This is mostly due to the health of the consumer offset topic.  replicatio

[jira] [Created] (KAFKA-7070) KafkaConsumer#committed might unexpectedly shift consumer offset

2018-06-18 Thread JIRA
Jan Lukavský created KAFKA-7070: --- Summary: KafkaConsumer#committed might unexpectedly shift consumer offset Key: KAFKA-7070 URL: https://issues.apache.org/jira/browse/KAFKA-7070 Project: Kafka

Re: [EXTERNAL] [DISCUSS] KIP-310: Add a Kafka Source Connector to Kafka Connect

2018-06-18 Thread McCaig, Rhys
Hi Stephane, Thanks for your feedback and apologies for the delay in my response. > Are there any performance benchmarks against Mirror Maker available? I'm > interested to know if this is more performant / scalable. > Regarding the implementation, here's some feedback: Currently I don’t have

Re: [DISCUSS] KIP-312: Add Overloaded StreamsBuilder Build Method to Accept java.util.Properties

2018-06-18 Thread Bill Bejeck
All, the discussion list for this proposed change has been quiet for a few days. If there are no changes or other proposals, I'll start a voting thread later today. Thanks, Bill On Wed, Jun 13, 2018 at 12:31 PM Guozhang Wang wrote: > Thanks for the explanation Bill. Makes sense to me. > > On T

[jira] [Resolved] (KAFKA-7069) AclCommand does not allow 'create' operation on 'topic'

2018-06-18 Thread Andy Coates (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andy Coates resolved KAFKA-7069. Resolution: Invalid > AclCommand does not allow 'create' operation on 'topic' > --

[jira] [Created] (KAFKA-7069) AclCommand does not allow 'create' operation on 'topic'

2018-06-18 Thread Andy Coates (JIRA)
Andy Coates created KAFKA-7069: -- Summary: AclCommand does not allow 'create' operation on 'topic' Key: KAFKA-7069 URL: https://issues.apache.org/jira/browse/KAFKA-7069 Project: Kafka Issue Type

Re: Error in Kafka Stream

2018-06-18 Thread Amandeep Singh
Hi Guozhang, The file system is XFS and the folder is not a temp folder. The issue goes away when I restart the streams. I forgot to mention i am running 3 multiple instances of consumer on 3 machines. Also, this issue seems to be reported by other users too: https://issues.apache.org/jira/browse

[jira] [Resolved] (KAFKA-7065) Quickstart tutorial fails because of missing brokers

2018-06-18 Thread Holger Brandl (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7065?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Holger Brandl resolved KAFKA-7065. -- Resolution: Invalid I misread the tutorial instructions. There was no bug. > Quickstart tutori

Re: [VOTE] KIP-280: Enhanced log compaction

2018-06-18 Thread Luís Cabral
Hi Ted / Guozhang / Matthias, @Ted: I've now added your argument to the "Rejected Alternatives" portion of the KIP. Please keep in mind that I would like to keep this as backwards compatible as possible, so a lot of decisions are inferred from that intent. @Guozhang: IMHO, adding expression ev

Build failed in Jenkins: kafka-trunk-jdk8 #2748

2018-06-18 Thread Apache Jenkins Server
See Changes: [github] MINOR: provide an example for deserialization exception handler (#5231) -- [...truncated 944.10 KB...] kafka.zookeeper.ZooKeeperClientTest > testBl