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/11/23 05:12:35 UTC

[jira] [Commented] (SAMZA-80) KV cache store fails with NPE

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

Chris Riccomini commented on SAMZA-80:
--------------------------------------

Ok, I don't have a patch for this yet, but this bug is painful. In CachedStore.scala, it appears that this cod is broken:

{noformat}
      if (found.dirty.prev == null) {
        this.dirty = found.dirty.next
        this.dirty.prev = null
      } else {
        found.dirty.remove
      }
{noformat}

In particular, the found.dirty.remove block never gets called. In particular, I set a break point when the NPE is thrown, and I took a look at this.dirty. Here's what I see:

{noformat}
this	CachedStore$$anonfun$flush$2  (id=59)	
	$outer	CachedStore<K,V>  (id=69)	
		bitmap$0	1	
		cacheSize	1000	
		grizzled$slf4j$Logging$$_logger	Logger  (id=104)	
		metrics	CachedStoreMetrics  (id=106)	
		org$apache$samza$storage$kv$CachedStore$$cache	CachedStore$$anon$1  (id=70)	
		org$apache$samza$storage$kv$CachedStore$$dirty	DoubleLinkedList<A>  (id=109)	
			elem	"remote-nav-foot" (id=369)	
			next	DoubleLinkedList<A>  (id=370)	
				elem	"pulse-editor-update-article-submit-internal" (id=372)	
				next	DoubleLinkedList<A>  (id=373)	
					elem	"pulse-editor-internal" (id=375)	
					next	DoubleLinkedList<A>  (id=376)	
						elem	"pulse-editor-article-promotions-page-internal" (id=378)	
						next	DoubleLinkedList<A>  (id=379)	
							elem	"profile_v2_treasury_landing" (id=381)	
							next	DoubleLinkedList<A>  (id=382)	
								elem	"profile_v2_megaphone_discovery" (id=384)	
								next	DoubleLinkedList<A>  (id=385)	
									elem	"profile_v2_activity_broadcasts_submit" (id=61)	
									next	DoubleLinkedList<A>  (id=387)	
										elem	"profile_v2_activity" (id=389)	
										next	DoubleLinkedList<A>  (id=390)	
											elem	"nprofile-edit" (id=392)	
											next	DoubleLinkedList<A>  (id=393)	
												elem	"remote-nav-foot" (id=395)	
												next	DoubleLinkedList<A>  (id=396)	
													elem	"quickhelp-search-catalog" (id=398)	
													next	DoubleLinkedList<A>  (id=399)	
													prev	null	
												prev	null	
											prev	null	
										prev	null	
									prev	null	
								prev	null	
							prev	null	
						prev	null	
					prev	null	
				prev	null	
			prev	null	
		org$apache$samza$storage$kv$CachedStore$$dirtyCount	5	
		store	SerializedKeyValueStore<K,V>  (id=111)	
		writeBatchSize	500	
	batch$1	ArrayList<E>  (id=90)	
{noformat}

Of particular note here is that EVERY prev variable is null (!!).

I added a debug statement next to found.dirty.remove(), and indeed this method is NEVER triggered. This results in totally broken dirty lists, since we treat every removed node as though it's head, even if it's not. This leads to, among other things, vastly different values between dirtyCount, and the number of elements in this.dirty.

I'm going to try and replicate this from a test.

> 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)