[
https://issues.apache.org/jira/browse/SAMZA-80?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13837981#comment-13837981
]
Chris Riccomini commented on SAMZA-80:
--------------------------------------
I kind of couldn't believe that Scala 2.8's double linked list is so broken. As
a sanity check, I ran this 2.8.1 code:
{code}
val tail = new scala.collection.mutable.DoubleLinkedList[Int]()
val zero = new scala.collection.mutable.DoubleLinkedList[Int](0, tail)
val one = new scala.collection.mutable.DoubleLinkedList[Int](1, zero)
val two = new scala.collection.mutable.DoubleLinkedList[Int](2, one)
System.err.println(two)
System.err.println(one.prev)
{code}
And got:
{noformat}
DoubleLinkedList(2, 1, 0)
null
{noformat}
The same code on 2.9.2 prints:
{noformat}
DoubleLinkedList(2, 1, 0)
DoubleLinkedList(2, 1, 0)
{noformat}
Scala 2.8 is definitely busted.
> 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
>
> Attachments: SAMZA-80.0.patch
>
>
> 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)