[
https://issues.apache.org/jira/browse/SAMZA-80?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13830567#comment-13830567
]
Chris Riccomini commented on SAMZA-80:
--------------------------------------
Sorry about all the typos above. Fried brain.
This bug exists only for Scala 2.8. I managed to reproduce this failure using:
{code}
@Test
def testRandomReadWriteRemove() {
// Make test deterministic by seeding the random number generator.
val rand = new Random(12345)
val something = b("a")
val keys = letters
.map(b(_))
.toArray
(0 until 100).foreach(loop => {
(0 until 30).foreach(i => {
val idx = rand.nextInt(keys.length)
val key = keys(idx)
val currentVal = store.get(key)
store.put(key, something)
})
val iterator = store.all
while (iterator.hasNext) {
store.delete(iterator.next.getKey)
}
iterator.close
})
}
{code}
To add to the fun, this only fails in Scala 2.8.
{noformat}
./gradlew -PscalaVersion=2.8.1 clean :samza-kv:test
-Dtest.single=TestKeyValueStores
{noformat}
But the the same test using:
{noformat}
./gradlew clean :samza-kv:test -Dtest.single=TestKeyValueStores
{noformat}
Does not fail. Sigh.
> KV cache store fails with NPE
> -----------------------------
>
> Key: SAMZA-80
> URL: https://issues.apache.org/jira/browse/SAMZA-80
> Project: Samza
> Issue Type: Bug
> Components: kv
> Affects Versions: 0.6.0
> Reporter: Chris Riccomini
> Assignee: Chris Riccomini
> Fix For: 0.7.0
>
>
> I came across a situation in which a StreamTask with a store will trigger an
> NPE in the caching class.
> {noformat}
> java.lang.NullPointerException
> at
> org.apache.samza.storage.kv.CachedStore$$anonfun$flush$2.apply(CachedStore.scala:144)
> at
> org.apache.samza.storage.kv.CachedStore$$anonfun$flush$2.apply(CachedStore.scala:142)
> at
> scala.collection.mutable.LinkedListLike$class.foreach(LinkedListLike.scala:111)
> at
> scala.collection.mutable.DoubleLinkedList.foreach(DoubleLinkedList.scala:41)
> at org.apache.samza.storage.kv.CachedStore.flush(CachedStore.scala:142)
> at
> org.apache.samza.storage.kv.CachedStore$$anon$1.removeEldestEntry(CachedStore.scala:65)
> at java.util.LinkedHashMap.addEntry(LinkedHashMap.java:410)
> at java.util.HashMap.put(HashMap.java:385)
> at org.apache.samza.storage.kv.CachedStore.get(CachedStore.scala:83)
> at
> org.apache.samza.storage.kv.KeyValueStorageEngine.get(KeyValueStorageEngine.scala:48)
> at com.linkedin.samza.example.StatefulTask.process(StatefulTask.java:51)
> at org.apache.samza.container.TaskInstance.process(TaskInstance.scala:168)
> at
> org.apache.samza.container.SamzaContainer.process(SamzaContainer.scala:563)
> at org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:455)
> at org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> {noformat}
--
This message was sent by Atlassian JIRA
(v6.1#6144)