[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2018-02-06 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16354546#comment-16354546
 ] 

Guozhang Wang commented on KAFKA-4750:
--

As a summary of this resolution:

{code}
Here is the new rule for handling nulls:
* in the interface store, put(key, null) are handled normally and value serde 
applied to null.
* in the inner most store, null bytes after serialization will always be 
treated as deletes.
* in the inner most store, range queries returning iterators should never 
return null bytes.
* in the interface store, if null bytes get returned in get(key), serde will be 
avoided and null object will be returned.
{code}

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>Priority: Major
>  Labels: newbie
> Fix For: 1.1.0
>
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2018-02-01 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16349240#comment-16349240
 ] 

ASF GitHub Bot commented on KAFKA-4750:
---

guozhangwang closed pull request #3430: KAFKA-4750: RocksDBStore always deletes 
null values
URL: https://github.com/apache/kafka/pull/3430
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
index 70345927f24..528da666e6e 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
@@ -245,7 +245,7 @@ public synchronized void put(K key, V value) {
 Objects.requireNonNull(key, "key cannot be null");
 validateStoreOpen();
 byte[] rawKey = serdes.rawKey(key);
-byte[] rawValue = serdes.rawValue(value);
+byte[] rawValue = value == null ? null : serdes.rawValue(value);
 putInternal(rawKey, rawValue);
 }
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreCustomNullSerdeTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreCustomNullSerdeTest.java
new file mode 100644
index 000..776747af211
--- /dev/null
+++ 
b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreCustomNullSerdeTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.test.MockProcessorContext;
+import org.apache.kafka.test.NoOpRecordCollector;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test checks RocksDBStore behaviour with serializer, which
+ * serializes null value into non-null byte array.
+ */
+public class RocksDBStoreCustomNullSerdeTest {
+private RocksDBStore subject;
+private MockProcessorContext context;
+
+@Before
+public void setUp() throws Exception {
+final Serializer serializer = new StringSerializer() {
+@Override
+public byte[] serialize(final String topic, final String data) {
+if (data == null) {
+return "null-encoding-that-is-not-just-'null'".getBytes();
+}
+return super.serialize(topic, data);
+}
+};
+final Serde serde = Serdes.serdeFrom(serializer, new 
StringDeserializer());
+subject = new RocksDBStore<>("test", serde, serde);
+context = new MockProcessorContext(
+TestUtils.tempDirectory(),
+serde,
+serde,
+new NoOpRecordCollector(),
+new ThreadCache("testCache", 0, new MockStreamsMetrics(new 
Metrics(;
+}
+
+@After
+public void tearDown() throws Exception {
+subject.close();
+}
+
+@Test
+public void shouldNotReturnDeletedInIterator() {
+subject.init(context, subject);
+subject.put("a", "1");
+subject.put("b", "2");
+subject.delete("a");
+final KeyValueIterator it = subject.all();
+while (it.hasNext()) {
+   

[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2018-02-01 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16349235#comment-16349235
 ] 

ASF GitHub Bot commented on KAFKA-4750:
---

guozhangwang opened a new pull request #4508: KAFKA-4750: Bypass null value and 
treat it as deletes
URL: https://github.com/apache/kafka/pull/4508
 
 
   * If the value is null, bypass the serde and treat it as deletes in the 
inner-most underlying store.
   * Update javadocs, add unit tests accordingly
   
   This is originally contributed by @evis.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>Priority: Major
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-07-24 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16098929#comment-16098929
 ] 

Guozhang Wang commented on KAFKA-4750:
--

[~evis] That is right, we can go with the first option above. Also as [~mjsax] 
mentioned we can enforce in the code that if the passed in value is `null` 
object for ser / bytes for deser, skip calling the serde and return the 
corresponding `null` bytes / object directly, besides stating it clearly in the 
javadoc.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-07-22 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16097368#comment-16097368
 ] 

Matthias J. Sax commented on KAFKA-4750:


IMHO, we can also improve the code by not calling the serializer in the first 
place if we get a `put(key, null)`. Thus, we can insure that `rawValue` will be 
`null` even if the serializer might not return `null`.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-07-22 Thread Evgeny Veretennikov (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16097357#comment-16097357
 ] 

Evgeny Veretennikov commented on KAFKA-4750:


So, to sum up discussion. I should close current PR and create new PR with 
javadoc changing, as [~guozhang] suggested. Am I right?

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-07-20 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16095776#comment-16095776
 ] 

Matthias J. Sax commented on KAFKA-4750:


I understand what you are saying, but encoding an empty collection into 
{{null}} would not work after all, as if we write this to the changelog, it 
might get delete during compaction resulting in potential data loss. Thus, 
Streams is limited by the underlying broker semantics and thus, users need to 
encode empty collections as not-null value anyway.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-07-20 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16094568#comment-16094568
 ] 

Matthias J. Sax commented on KAFKA-4750:


As mentioned above. I would go with option (1) as null delete semantics also 
align with changelog delete semantics. Also, if we would allow to put {{null}} 
as regular value, we have the same issue as Java has in general: if a {{get}} 
returns {{null}} it's unclear if {{null}} is a regular value or the key is just 
not there. Within Java (eg. {{HashMap}}) you still can check with 
{{containsKey}} but we don't have an API for this on the stores.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-07-19 Thread Bill Bejeck (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16093595#comment-16093595
 ] 

Bill Bejeck commented on KAFKA-4750:


Thanks for the clarification [~guozhang], my inclination is to go with the 
second option, as that seem more intuitive.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.10.2.1, 0.11.0.0
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-27 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16065155#comment-16065155
 ] 

Guozhang Wang commented on KAFKA-4750:
--

[~evis] Inside RocksDB store, after the serialization, if we get "null" byte 
arrays (NOTE it is not "null" object that gets passed into the API) then we 
should always treat it as a delete call; i.e. the current implementation inside 
RocksDB is ok:

{code}
private void putInternal(byte[] rawKey, byte[] rawValue) {
if (rawValue == null) {
try {
db.delete(wOptions, rawKey);
} catch (RocksDBException e) {
   ...
}
} else {
try {
db.put(wOptions, rawKey, rawValue);
} catch (RocksDBException e) {
...
}
}
}
{code}

The question is, on the API layer do we want to enforce "null" object to 
indicate deletion as well. Currently we are a bit vague in this, I was 
proposing two options and make it clear:

1) Clarify in javadoc that null value in {{put(key, value)}} indicates 
deletion; if it is "null" object by-pass the serde and send "null" bytes 
directly into inner functions and vice verse for deserialization; do not 
enforce user customized serdes how to handle null values since we are not going 
to call them with null values any more.

2) Do NOT enforce in java doc that null value in {{put(key, value)}} indicates 
deletion; implement all {{delete(key)}} call directly throughout all the layers 
of stores instead of calling {{put(key, null)}}; recommend user customized 
serdes to handle null values themselves.

I am a bit inclined to the second option, and [~mjsax] seem to be favoring the 
first option. And I'd like to hear see how others think.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-26 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16063956#comment-16063956
 ] 

Matthias J. Sax commented on KAFKA-4750:


I think [~damianguy] or [~enothereska] can comment best in this. AFAIK, we use 
{{put(key,null)}} with delete-semantics all over the place. Also for {{KTable}} 
caches. As it align with changelog delete semantics I also think it does make 
sense to keep it this way. I would rather educate user that plug in Serde to 
not return {{null}} if input is not {{null}}. We can also add checks to all 
{{Serde}} calls: (1) never call Serde for {{null}} as we know it must be 
{{null}} anyway (2) if we call Serde with not-null, make sure it does not 
return {{null}} -- otherwise throw exception.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-26 Thread Evgeny Veretennikov (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16062575#comment-16062575
 ] 

Evgeny Veretennikov commented on KAFKA-4750:


If we invoke {{store.put(key, value)}}, and serdes returns null for value, 
shouldn't we throw NPE instead of deleting key from store? It seems 
counter-intuitive, that value here can be null, since null indicates, that no 
value found:

{code:java}
Object value = ???; // some value serialized to null by serde
store.put(key, value);
Object value = store.get(key); // returns null, that seems like no value found
{code}

Throwing exception inside {{put()}} prevents from potential data loss.

By the way, why does {{KeyValueStore.put()}} method allows value to be null?

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-25 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16062423#comment-16062423
 ] 

Matthias J. Sax commented on KAFKA-4750:


Sounds good to me.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-25 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16062384#comment-16062384
 ] 

Guozhang Wang commented on KAFKA-4750:
--

I think the Streams library should not assume that either 1) the serde will 
always return {{null}} if the input is {{null}}, or 2) the serde will never 
return {{null}} if the input is not {{null}}. On the other hand, since we are 
treating {{null}} to indicate deletion specifically, checking null values 
before calling the serde makes sense to me.



> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-24 Thread Matthias J. Sax (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16062112#comment-16062112
 ] 

Matthias J. Sax commented on KAFKA-4750:


Your observation is correct. I simplified the call chain. The question than is, 
it is a bug in the {{Serde}} that it does not return {{null}} if value is 
{{null}}? Or should we just make Streams itself more robust and change {{byte[] 
rawValue = serdes.rawValue(value);}} to {{byte[] rawValue = value == null ? 
null : serdes.rawValue(value);}} ?

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
> Attachments: DeleteTest.java
>
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-24 Thread Evgeny Veretennikov (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16061898#comment-16061898
 ] 

Evgeny Veretennikov commented on KAFKA-4750:


[~mjsax], are you sure about call chain in {{RocksDBStore}}? Actually it is 
{{delete(key) -> put(key, null) -> putInternal(serdes.rawKey(key), 
serdes.rawValue(value)}}. Check {{RocksDBStore.put()}} method:

{code:java}
public synchronized void put(K key, V value) {
Objects.requireNonNull(key, "key cannot be null");
validateStoreOpen();
byte[] rawKey = serdes.rawKey(key);
byte[] rawValue = serdes.rawValue(value);
putInternal(rawKey, rawValue);
}
{code}

So, {{delete()}} doesn't actually delete key-value pair if 
{{serdes.rawValue(null)}} doesn't return {{null}}.

For example, look at attached test. Here store uses custom serde, which 
serializes null into not-null value {{"123".getBytes()}}. Test fails with 
current {{RocksDBStore.delete()}} implementation and succeeds with such change:

{noformat}
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java
@@ -301,7 +301,9 @@ public class RocksDBStore implements KeyValueStore {
 public synchronized V delete(K key) {
 Objects.requireNonNull(key, "key cannot be null");
 V value = get(key);
-put(key, null);
+validateStoreOpen();
+byte[] rawKey = serdes.rawKey(key);
+putInternal(rawKey, null);
 return value;
 }
{noformat}

Though, in {{ChangeLoggingKeyValueBytesStore}} this problem really doesn't 
appear, that was my mistake.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1, 0.11.0.0, 0.10.2.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (KAFKA-4750) KeyValueIterator returns null values

2017-06-23 Thread Evgeny Veretennikov (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-4750?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16061243#comment-16061243
 ] 

Evgeny Veretennikov commented on KAFKA-4750:


I guess, problem lies in {{KeyValueStore.delete()}} implementations, which 
invoke {{put(key, null)}} instead of actual deleting key-value pair. 
{{RocksDBStore}} and {{ChangeLoggingKeyValueBytesStore}} are implemented such 
way. So, I suggest just to fix these two {{delete()}} implementations.

Though, I don't quite understand, why did this scenario worked in 0.10.0.0... 
These {{delete()}} methods didn't change in past.

> KeyValueIterator returns null values
> 
>
> Key: KAFKA-4750
> URL: https://issues.apache.org/jira/browse/KAFKA-4750
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 0.10.1.1
>Reporter: Michal Borowiecki
>Assignee: Evgeny Veretennikov
>  Labels: newbie
>
> The API for ReadOnlyKeyValueStore.range method promises the returned iterator 
> will not return null values. However, after upgrading from 0.10.0.0 to 
> 0.10.1.1 we found null values are returned causing NPEs on our side.
> I found this happens after removing entries from the store and I found 
> resemblance to SAMZA-94 defect. The problem seems to be as it was there, when 
> deleting entries and having a serializer that does not return null when null 
> is passed in, the state store doesn't actually delete that key/value pair but 
> the iterator will return null value for that key.
> When I modified our serilizer to return null when null is passed in, the 
> problem went away. However, I believe this should be fixed in kafka streams, 
> perhaps with a similar approach as SAMZA-94.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)