You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by "Chris Riccomini (JIRA)" <ji...@apache.org> on 2013/08/28 00:09:52 UTC

[jira] [Commented] (SAMZA-45) NPE in linked list in samza-kv's CachedStore

    [ https://issues.apache.org/jira/browse/SAMZA-45?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13751781#comment-13751781 ] 

Chris Riccomini commented on SAMZA-45:
--------------------------------------

Seems like this fixes it:

-    else
-      found.dirty.remove()
+    else {
+      // If we are removing the head of the list, move the head to the next element.
+      if(found.dirty.prev == null) {
+        this.dirty = found.dirty.next
+        this.dirty.prev = null
+      } else {
+        found.dirty.remove
+      }
+    }

Special casing the head of the list, and manually deleting it. Not sure if this is safe.
                
> NPE in linked list in samza-kv's CachedStore
> --------------------------------------------
>
>                 Key: SAMZA-45
>                 URL: https://issues.apache.org/jira/browse/SAMZA-45
>             Project: Samza
>          Issue Type: Bug
>          Components: kv
>    Affects Versions: 0.6.0
>            Reporter: Chris Riccomini
>            Assignee: Jay Kreps
>             Fix For: 0.7.0
>
>
> Occasionally, I get an NPE from Samza when a job that uses samsa-kv. Here's a full stack trace.
> {noformat}
> java.lang.NullPointerException
> 	at scala.collection.mutable.DoubleLinkedListLike$class.remove(DoubleLinkedListLike.scala:55)
> 	at scala.collection.mutable.DoubleLinkedList.remove(DoubleLinkedList.scala:41)
> 	at org.apache.samza.storage.kv.CachedStore.put(CachedStore.scala:96)
> 	at org.apache.samza.storage.kv.KeyValueStorageEngine.put(KeyValueStorageEngine.scala:46)
> 	at org.apache.samza.test.integration.StatePerfTestTask.process(StatePerfTestTask.java:24)
> 	at org.apache.samza.container.TaskInstance.process(TaskInstance.scala:162)
> 	at org.apache.samza.container.SamzaContainer.process(SamzaContainer.scala:540)
> 	at org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:436)
> 	at org.apache.samza.job.local.ThreadJob$$anon$1.run(ThreadJob.scala:42)
> {noformat}
> I saved my Kafka logs (/tmp/kafka-logs) that trigger this (using StatePerfTask) running on a Kafka grid setup using hello-samza's bin/grid script. The logs are a bit too large (~75 megs) to attach to a JIRA.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira