Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Assert failure in block.DecrRef #1350

Closed
jarifibrahim opened this issue May 28, 2020 · 7 comments
Closed

Assert failure in block.DecrRef #1350

jarifibrahim opened this issue May 28, 2020 · 7 comments
Assignees
Labels
area/crash This issue causes a panic or some other of exception that causes a crash. help wanted Please help us fix this! kind/bug Something is broken. priority/P1 Serious issue that requires eventual attention (can wait a bit) status/accepted We accept to investigate or work on it.

Comments

@jarifibrahim
Copy link
Contributor

What version of Go are you using (go version)?

go 1.13

What version of Badger are you using?

6eaa500

Steps to Reproduce the issue

  1. Run dgraph on version dgraph-io/dgraph@cc97957 and apply the following patch
diff --git a/worker/server_state.go b/worker/server_state.go
index fe5d7ffad..f27065e10 100644
--- a/worker/server_state.go
+++ b/worker/server_state.go
@@ -71,13 +71,13 @@ func setBadgerOptions(opt badger.Options) badger.Options {
 	// Do not load bloom filters on DB open.
 	opt.LoadBloomsOnOpen = false
 
-	glog.Infof("Setting Badger Compression Level: %d", Config.BadgerCompressionLevel)
+	glog.Infof("Setting Badger C:mpression Level: %d", Config.BadgerCompressionLevel)
 	// Default value of badgerCompressionLevel is 3 so compression will always
 	// be enabled, unless it is explicitly disabled by setting the value to 0.
 	if Config.BadgerCompressionLevel != 0 {
 		// By default, compression is disabled in badger.
-		opt.Compression = options.ZSTD
-		opt.ZSTDCompressionLevel = Config.BadgerCompressionLevel
+		// 	opt.Compression = options.ZSTD
+		// 	opt.ZSTDCompressionLevel = Config.BadgerCompressionLevel
 	}
 
 	glog.Infof("Setting Badger table load option: %s", Config.BadgerTables)
  1. Run flock against 6 node dgraph cluster

What did you do?

Run queries and mutation on dgraph.

What did you expect to see?

I expected no crash.

What did you see instead?

alpha3    | I0528 16:42:09.759548      15 mvcc.go:80] Rolled up 118000 keys
alpha2    | I0528 16:42:10.020588      16 mvcc.go:80] Rolled up 115000 keys
alpha1    | I0528 16:42:12.529281      18 mvcc.go:80] Rolled up 116000 keys
alpha2    | 2020/05/28 16:42:12 Assert failed
alpha2    | github.com/dgraph-io/badger/v2/y.AssertTrue
alpha2    | 	/home/ibrahim/Projects/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200522174526-6eaa5009af27/y/error.go:55
alpha2    | github.com/dgraph-io/badger/v2/table.(*block).decrRef
alpha2    | 	/home/ibrahim/Projects/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200522174526-6eaa5009af27/table/table.go:229
alpha2    | github.com/dgraph-io/badger/v2/table.BlockEvictHandler
alpha2    | 	/home/ibrahim/Projects/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200522174526-6eaa5009af27/table/table.go:175
alpha2    | github.com/dgraph-io/badger/v2.Open.func4
alpha2    | 	/home/ibrahim/Projects/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200522174526-6eaa5009af27/db.go:303
alpha2    | github.com/dgraph-io/ristretto.(*Cache).processItems
alpha2    | 	/home/ibrahim/Projects/go/pkg/mod/github.com/dgraph-io/ristretto@v0.0.2/cache.go:315
alpha2    | runtime.goexit
alpha2    | 	/usr/local/go/src/runtime/asm_amd64.s:1357
alpha3    | W0528 16:42:12.671440      15 pool.go:254] Connection lost with alpha2:7182. Error: rpc error: code = Unavailable desc = transport is closing
zero3     | W0528 16:42:12.671437      17 pool.go:254] Connection lost with alpha2:7182. Error: rpc error: code = Unavailable desc = transport is closing
alpha1    | W0528 16:42:12.672130      18 pool.go:254] Connection lost with alpha2:7182. Error: rpc error: code = Unavailable desc = transport is closing
zero2     | W0528 16:42:12.672152      16 pool.go:254] Connection lost with alpha2:7182. Error: rpc error: code = Unavailable desc = transport is closing
alpha1    | W0528 16:42:12.691573      18 node.go:420] Unable to send message to peer: 0x2. Error: EOF
alpha3    | I0528 16:42:12.989356      15 mvcc.go:80] Rolled up 119000 keys
alpha2 exited with code 1
@jarifibrahim jarifibrahim added kind/bug Something is broken. priority/P1 Serious issue that requires eventual attention (can wait a bit) status/accepted We accept to investigate or work on it. area/crash This issue causes a panic or some other of exception that causes a crash. labels May 28, 2020
@darkn3rd
Copy link

darkn3rd commented Jun 16, 2020

When running flock for several days with v20.07.0-beta.Jun15, I ran into this on alpha1:

2020/06/16 00:54:49 Assert failed
github.com/dgraph-io/badger/v2/y.AssertTrue
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200528205344-e7b6e76f96e8/y/error.go:55
github.com/dgraph-io/badger/v2/table.(*block).decrRef
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200528205344-e7b6e76f96e8/table/table.go:244
github.com/dgraph-io/badger/v2/table.BlockEvictHandler
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200528205344-e7b6e76f96e8/table/table.go:190
github.com/dgraph-io/badger/v2.Open.func4
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200528205344-e7b6e76f96e8/db.go:303
github.com/dgraph-io/ristretto.(*Cache).processItems
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/ristretto@v0.0.2/cache.go:315
runtime.goexit
	/usr/local/go/src/runtime/asm_amd64.s:1373

@jarifibrahim
Copy link
Contributor Author

@darkn3rd do you know how long flock was running for?

@darkn3rd
Copy link

@jarifibrahim I'm unsure about the last run, I think for a few hours. Recently I tried to run with the existing data directories, but upgraded server, and I ran into the problem. It took 20 minutes to reproduce.

Logs

[Decoder]: Using assembly version of decoder
[Sentry] 2020/06/25 22:05:07 Integration installed: ContextifyFrames
[Sentry] 2020/06/25 22:05:07 Integration installed: Environment
[Sentry] 2020/06/25 22:05:07 Integration installed: Modules
[Sentry] 2020/06/25 22:05:07 Integration installed: IgnoreErrors
[Decoder]: Using assembly version of decoder
[Sentry] 2020/06/25 22:05:08 Integration installed: ContextifyFrames
[Sentry] 2020/06/25 22:05:08 Integration installed: Environment
[Sentry] 2020/06/25 22:05:08 Integration installed: Modules
[Sentry] 2020/06/25 22:05:08 Integration installed: IgnoreErrors
I0625 22:05:08.465587      15 sentry_integration.go:48] This instance of Dgraph will send anonymous reports of panics back to Dgraph Labs via Sentry. No confidential information is sent. These reports help improve Dgraph. To opt-out, restart your instance with the --enable_sentry=false flag. For more info, see https://dgraph.io/docs/howto/#data-handling.
I0625 22:05:08.466843      15 util_ee.go:126] KeyReader instantiated of type <nil>
I0625 22:05:08.643733      15 init.go:99]

Dgraph version   : v20.07.0-beta.Jun22-7-g73a9ddd80
Dgraph SHA-256   : e685d19897fd1b9784a14f14dc0d805f8011d2dbc92ca444ad4a2b7adca6523e
Commit SHA-1     : 73a9ddd80
Commit timestamp : 2020-06-25 13:24:40 -0700
Branch           : release/v20.07
Go version       : go1.14.2

For Dgraph official documentation, visit https://dgraph.io/docs/.
For discussions about Dgraph     , visit https://discuss.dgraph.io.
To say hi to the community       , visit https://dgraph.slack.com.

Licensed variously under the Apache Public License 2.0 and Dgraph Community License.
Copyright 2015-2020 Dgraph Labs, Inc.


I0625 22:05:08.643771      15 run.go:641] x.Config: {PortOffset:0 QueryEdgeLimit:1000000 NormalizeNodeLimit:10000 PollInterval:1s}
I0625 22:05:08.643797      15 run.go:642] x.WorkerConfig: {ExportPath:export NumPendingProposals:256 Tracing:0.01 MyAddr:alpha1:7080 ZeroAddr:[zero1:5080] RaftId:0 WhiteListedIPRanges:[] MaxRetries:-1 StrictMutations:false AclEnabled:false AbortOlderThan:5m0s SnapshotAfter:10000 ProposedGroupId:0 StartTime:2020-06-25 22:05:08.104203835 +0000 UTC m=+0.014817380 LudicrousMode:false EncryptionKey:**** LogRequest:0}
I0625 22:05:08.643831      15 run.go:643] worker.Config: {PostingDir:p BadgerTables:mmap BadgerVlog:mmap BadgerCompressionLevel:3 WALDir:w MutationsMode:0 AuthToken: AllottedMemory:1024 HmacSecret:**** AccessJwtTtl:0s RefreshJwtTtl:0s AclRefreshInterval:0s}
I0625 22:05:08.643898      15 server_state.go:80] Setting Badger Compression Level: 3
I0625 22:05:08.643903      15 server_state.go:89] Setting Badger table load option: mmap
I0625 22:05:08.643907      15 server_state.go:101] Setting Badger value log load option: mmap
I0625 22:05:08.643917      15 server_state.go:146] Opening write-ahead log BadgerDB with options: {Dir:w ValueDir:w SyncWrites:false TableLoadingMode:1 ValueLogLoadingMode:2 NumVersionsToKeep:1 ReadOnly:false Truncate:true Logger:0x2b78fd0 Compression:2 InMemory:false MaxTableSize:67108864 LevelSizeMultiplier:10 MaxLevels:7 ValueThreshold:1048576 NumMemtables:5 BlockSize:4096 BloomFalsePositive:0.01 KeepL0InMemory:false MaxCacheSize:10485760 MaxBfCacheSize:0 LoadBloomsOnOpen:false NumLevelZeroTables:5 NumLevelZeroTablesStall:10 LevelOneSize:268435456 ValueLogFileSize:1073741823 ValueLogMaxEntries:10000 NumCompactors:2 CompactL0OnClose:true LogRotatesToFlush:2 ZSTDCompressionLevel:3 VerifyValueChecksum:false EncryptionKey:[] EncryptionKeyRotationDuration:240h0m0s BypassLockGuard:false ChecksumVerificationMode:0 DetectConflicts:false KeepBlockIndicesInCache:false KeepBlocksInCache:false managedTxns:false maxBatchCount:0 maxBatchSize:0}
I0625 22:05:08.655763      15 log.go:34] All 2 tables opened in 5ms
I0625 22:05:08.658771      15 log.go:34] Replaying file id: 4501 at offset: 4587032
I0625 22:05:08.658851      15 log.go:34] Replay took: 56.757µs
I0625 22:05:08.658888      15 log.go:34] Replaying file id: 4502 at offset: 0
I0625 22:05:08.682758      15 log.go:34] Replay took: 23.860349ms
I0625 22:05:08.682852      15 log.go:34] Replaying file id: 4503 at offset: 0
I0625 22:05:08.685928      15 log.go:34] Replay took: 3.065584ms
I0625 22:05:08.686517      15 server_state.go:80] Setting Badger Compression Level: 3
I0625 22:05:08.686536      15 server_state.go:89] Setting Badger table load option: mmap
I0625 22:05:08.686542      15 server_state.go:101] Setting Badger value log load option: mmap
I0625 22:05:08.686550      15 server_state.go:170] Opening postings BadgerDB with options: {Dir:p ValueDir:p SyncWrites:false TableLoadingMode:2 ValueLogLoadingMode:2 NumVersionsToKeep:2147483647 ReadOnly:false Truncate:true Logger:0x2b78fd0 Compression:2 InMemory:false MaxTableSize:67108864 LevelSizeMultiplier:10 MaxLevels:7 ValueThreshold:1024 NumMemtables:5 BlockSize:4096 BloomFalsePositive:0.01 KeepL0InMemory:false MaxCacheSize:1073741824 MaxBfCacheSize:524288000 LoadBloomsOnOpen:false NumLevelZeroTables:5 NumLevelZeroTablesStall:10 LevelOneSize:268435456 ValueLogFileSize:1073741823 ValueLogMaxEntries:1000000 NumCompactors:2 CompactL0OnClose:true LogRotatesToFlush:2 ZSTDCompressionLevel:3 VerifyValueChecksum:false EncryptionKey:[] EncryptionKeyRotationDuration:240h0m0s BypassLockGuard:false ChecksumVerificationMode:0 DetectConflicts:false KeepBlockIndicesInCache:true KeepBlocksInCache:true managedTxns:false maxBatchCount:0 maxBatchSize:0}
I0625 22:05:10.408685      15 log.go:34] All 297 tables opened in 1.664s
I0625 22:05:10.462026      15 log.go:34] Replaying file id: 442 at offset: 50684148
I0625 22:05:10.462060      15 log.go:34] Replay took: 4.023µs
I0625 22:05:10.463123      15 groups.go:107] Current Raft Id: 0x1
I0625 22:05:10.463419      15 worker.go:97] Worker listening at address: [::]:7080
I0625 22:05:10.464362      15 run.go:513] Bringing up GraphQL HTTP API at 0.0.0.0:8080/graphql
I0625 22:05:10.464388      15 run.go:514] Bringing up GraphQL HTTP admin API at 0.0.0.0:8080/admin
I0625 22:05:10.464428      15 run.go:546] gRPC server started.  Listening on port 9080
I0625 22:05:10.464437      15 run.go:547] HTTP server started.  Listening on port 8080
I0625 22:05:10.563744      15 pool.go:160] CONNECTING to zero1:5080
I0625 22:05:14.568693      15 groups.go:135] Connected to group zero. Assigned group: 0
I0625 22:05:14.568736      15 groups.go:137] Raft Id after connection to Zero: 0x1
I0625 22:05:14.568953      15 draft.go:210] Node ID: 0x1 with GroupID: 1
I0625 22:05:14.569006      15 node.go:148] Setting raft.Config to: &{ID:1 peers:[] learners:[] ElectionTick:20 HeartbeatTick:1 Storage:0xc0825f1c80 Applied:16426094 MaxSizePerMsg:262144 MaxCommittedSizePerReady:67108864 MaxUncommittedEntriesSize:0 MaxInflightMsgs:256 CheckQuorum:false PreVote:true ReadOnlyOption:0 Logger:0x2b78fd0 DisableProposalForwarding:false}
I0625 22:05:14.569018      15 pool.go:160] CONNECTING to alpha1:7080
W0625 22:05:14.571027      15 pool.go:254] Connection lost with alpha1:7080. Error: rpc error: code = Unknown desc = No node has been set up yet
I0625 22:05:14.571758      15 node.go:306] Found Snapshot.Metadata: {ConfState:{Nodes:[1] Learners:[] XXX_unrecognized:[]} Index:16426094 Term:3 XXX_unrecognized:[]}
I0625 22:05:14.571788      15 node.go:317] Found hardstate: {Term:4 Vote:1 Commit:16427462 XXX_unrecognized:[]}
I0625 22:05:14.580827      15 node.go:326] Group 1 found 1368 entries
I0625 22:05:14.580854      15 draft.go:1595] Restarting node for group: 1
I0625 22:05:14.580880      15 node.go:185] Setting conf state to nodes:1
I0625 22:05:14.581430      15 log.go:34] 1 became follower at term 4
I0625 22:05:14.581949      15 log.go:34] newRaft 1 [peers: [1], term: 4, commit: 16427462, applied: 16426094, lastindex: 16427462, lastterm: 4]
I0625 22:05:14.581983      15 draft.go:160] Operation started with id: opRollup
I0625 22:05:14.581995      15 groups.go:155] Server is ready
I0625 22:05:14.582092      15 groups.go:790] Got address of a Zero leader: zero1:5080
I0625 22:05:14.582178      15 draft.go:1001] Found Raft progress: 16427462
I0625 22:05:14.582318      15 groups.go:804] Starting a new membership stream receive from zero1:5080.
I0625 22:05:14.582906      15 groups.go:821] Received first state update from Zero: counter:10916397 groups:<key:1 value:<members:<key:1 value:<id:1 group_id:1 addr:"alpha1:7080" leader:true last_update:1592864399 > > tablets:<key:"author" value:<group_id:1 predicate:"author" space:16866422277 > > tablets:<key:"created_at" value:<group_id:1 predicate:"created_at" space:324077075750 > > tablets:<key:"description" value:<group_id:1 predicate:"description" space:102979608756 > > tablets:<key:"dgraph.graphql.schema" value:<group_id:1 predicate:"dgraph.graphql.schema" > > tablets:<key:"dgraph.graphql.xid" value:<group_id:1 predicate:"dgraph.graphql.xid" > > tablets:<key:"dgraph.type" value:<group_id:1 predicate:"dgraph.type" space:1827482254 > > tablets:<key:"followers_count" value:<group_id:1 predicate:"followers_count" space:411606143 > > tablets:<key:"friends_count" value:<group_id:1 predicate:"friends_count" space:411614913 > > tablets:<key:"hashtags" value:<group_id:1 predicate:"hashtags" space:2479493467 > > tablets:<key:"id_str" value:<group_id:1 predicate:"id_str" space:4374722908 > > tablets:<key:"mention" value:<group_id:1 predicate:"mention" space:16407742505 > > tablets:<key:"message" value:<group_id:1 predicate:"message" space:11805860031 > > tablets:<key:"profile_banner_url" value:<group_id:1 predicate:"profile_banner_url" space:135114300150 > > tablets:<key:"profile_image_url" value:<group_id:1 predicate:"profile_image_url" space:1102085209 > > tablets:<key:"retweet" value:<group_id:1 predicate:"retweet" space:3307481423 > > tablets:<key:"screen_name" value:<group_id:1 predicate:"screen_name" space:1391134659 > > tablets:<key:"urls" value:<group_id:1 predicate:"urls" > > tablets:<key:"user_id" value:<group_id:1 predicate:"user_id" space:2286473313 > > tablets:<key:"user_name" value:<group_id:1 predicate:"user_name" space:5515738130 > > tablets:<key:"verified" value:<group_id:1 predicate:"verified" > > snapshot_ts:21810505 checksum:17475820780574559673 > > zeros:<key:1 value:<id:1 addr:"zero1:5080" leader:true > > maxLeaseId:19540000 maxTxnTs:21820000 maxRaftId:1 cid:"fabd9a37-66ab-408b-88f3-ec1c52ea61d2" license:<maxNodes:18446744073709551615 expiryTs:1595185409 enabled:true >
I0625 22:05:17.682737      15 log.go:34] 1 is starting a new election at term 4
I0625 22:05:17.682782      15 log.go:34] 1 became pre-candidate at term 4
I0625 22:05:17.682788      15 log.go:34] 1 received MsgPreVoteResp from 1 at term 4
I0625 22:05:17.683104      15 log.go:34] 1 became candidate at term 5
I0625 22:05:17.683119      15 log.go:34] 1 received MsgVoteResp from 1 at term 5
I0625 22:05:17.683745      15 log.go:34] 1 became leader at term 5
I0625 22:05:17.683767      15 log.go:34] raft.node: 1 elected leader 1 at term 5
I0625 22:05:18.582372      15 groups.go:873] Leader idx=0x1 of group=1 is connecting to Zero for txn updates
I0625 22:05:18.582425      15 groups.go:882] Got Zero leader: zero1:5080
I0625 22:05:18.731910      15 admin.go:661] No GraphQL schema in Dgraph; serving empty GraphQL API
I0625 22:11:47.818459      15 server.go:137] Received ALTER op: schema:"\n\t\ttype Tweet {\n\t\t\tid_str: string\n\t\t\tcreated_at: dateTime\n\t\t\tmessage: string\n\t\t\turls: [string]\n\t\t\thashtags: [string]\n\t\t\tauthor: User\n\t\t\tmention: [User]\n\t\t\tretweet: bool\n\t\t}\n\t\t\n\t\ttype User {\n\t\t\tuser_id\n\t\t\tuser_name\n\t\t\tscreen_name\n\t\t\tdescription\n\t\t\tfriends_count\n\t\t\tfollowers_count\n\t\t\tverified\n\t\t\tprofile_banner_url\n\t\t\tprofile_image_url\n\t\t}\n\n\t\tuser_id: string @index(exact) @upsert .\n\t\tuser_name: string @index(hash) .\n\t\tscreen_name: string @index(term) .\n\t\tdescription: string .\n\t\tfriends_count: int .\n\t\tfollowers_count: int .\n\t\tverified: bool .\n\t\tprofile_banner_url: string .\n\t\tprofile_image_url: string .\n\t\tid_str: string @index(exact) @upsert .\n\t\tcreated_at: dateTime @index(hour) .\n\t\tmessage: string .\n\t\turls: [string] .\n\t\thashtags: [string] @index(exact) .\n\t\tauthor: uid @count @reverse .\n\t\tmention: [uid] @reverse .\n\t\tretweet: bool .\n\t"
I0625 22:11:47.818505      15 server.go:1161] Got Alter request from "172.18.0.1:56406"
W0625 22:11:47.819350      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field id_str which will be ignored.
W0625 22:11:47.819376      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field created_at which will be ignored.
W0625 22:11:47.819390      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field message which will be ignored.
W0625 22:11:47.819398      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field urls which will be ignored.
W0625 22:11:47.819406      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field hashtags which will be ignored.
W0625 22:11:47.819413      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field author which will be ignored.
W0625 22:11:47.819420      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field mention which will be ignored.
W0625 22:11:47.819428      15 parse.go:405] Type declaration for type Tweet includes deprecated information about field type for field retweet which will be ignored.
I0625 22:11:47.819472      15 server.go:299] Got schema: &{Preds:[predicate:"user_id" value_type:STRING directive:INDEX tokenizer:"exact" upsert:true  predicate:"user_name" value_type:STRING directive:INDEX tokenizer:"hash"  predicate:"screen_name" value_type:STRING directive:INDEX tokenizer:"term"  predicate:"description" value_type:STRING  predicate:"friends_count" value_type:INT  predicate:"followers_count" value_type:INT  predicate:"verified" value_type:BOOL  predicate:"profile_banner_url" value_type:STRING  predicate:"profile_image_url" value_type:STRING  predicate:"id_str" value_type:STRING directive:INDEX tokenizer:"exact" upsert:true  predicate:"created_at" value_type:DATETIME directive:INDEX tokenizer:"hour"  predicate:"message" value_type:STRING  predicate:"urls" value_type:STRING list:true  predicate:"hashtags" value_type:STRING directive:INDEX tokenizer:"exact" list:true  predicate:"author" value_type:UID directive:REVERSE count:true  predicate:"mention" value_type:UID directive:REVERSE list:true  predicate:"retweet" value_type:BOOL ] Types:[type_name:"Tweet" fields:<predicate:"id_str" > fields:<predicate:"created_at" > fields:<predicate:"message" > fields:<predicate:"urls" > fields:<predicate:"hashtags" > fields:<predicate:"author" > fields:<predicate:"mention" > fields:<predicate:"retweet" >  type_name:"User" fields:<predicate:"user_id" > fields:<predicate:"user_name" > fields:<predicate:"screen_name" > fields:<predicate:"description" > fields:<predicate:"friends_count" > fields:<predicate:"followers_count" > fields:<predicate:"verified" > fields:<predicate:"profile_banner_url" > fields:<predicate:"profile_image_url" > ]}
I0625 22:11:47.824165      15 draft.go:160] Operation started with id: opIndexing
I0625 22:11:47.824375      15 draft.go:115] Operation completed with id: opRollup
I0625 22:11:47.824842      15 draft.go:115] Operation completed with id: opIndexing
I0625 22:11:47.824936      15 server.go:318] ALTER op: schema:"\n\t\ttype Tweet {\n\t\t\tid_str: string\n\t\t\tcreated_at: dateTime\n\t\t\tmessage: string\n\t\t\turls: [string]\n\t\t\thashtags: [string]\n\t\t\tauthor: User\n\t\t\tmention: [User]\n\t\t\tretweet: bool\n\t\t}\n\t\t\n\t\ttype User {\n\t\t\tuser_id\n\t\t\tuser_name\n\t\t\tscreen_name\n\t\t\tdescription\n\t\t\tfriends_count\n\t\t\tfollowers_count\n\t\t\tverified\n\t\t\tprofile_banner_url\n\t\t\tprofile_image_url\n\t\t}\n\n\t\tuser_id: string @index(exact) @upsert .\n\t\tuser_name: string @index(hash) .\n\t\tscreen_name: string @index(term) .\n\t\tdescription: string .\n\t\tfriends_count: int .\n\t\tfollowers_count: int .\n\t\tverified: bool .\n\t\tprofile_banner_url: string .\n\t\tprofile_image_url: string .\n\t\tid_str: string @index(exact) @upsert .\n\t\tcreated_at: dateTime @index(hour) .\n\t\tmessage: string .\n\t\turls: [string] .\n\t\thashtags: [string] @index(exact) .\n\t\tauthor: uid @count @reverse .\n\t\tmention: [uid] @reverse .\n\t\tretweet: bool .\n\t"  done
I0625 22:11:57.825132      15 draft.go:160] Operation started with id: opRollup
W0625 22:13:48.497932      15 draft.go:1222] Raft.Ready took too long to process: Timer Total: 216ms. Breakdown: [{sync 216ms} {disk 0s} {proposals 0s} {advance 0s}] Num entries: 1. MustSync: true
W0625 22:13:50.144252      15 draft.go:1222] Raft.Ready took too long to process: Timer Total: 230ms. Breakdown: [{sync 230ms} {disk 0s} {proposals 0s} {advance 0s}] Num entries: 1. MustSync: true
I0625 22:14:15.184530      15 draft.go:567] Creating snapshot at index: 16438229. ReadTs: 21834600.
2020/06/25 22:16:03 Assert failed
github.com/dgraph-io/badger/v2/y.AssertTrue
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200615081930-c45d966681d4/y/error.go:55
github.com/dgraph-io/badger/v2/table.(*block).decrRef
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200615081930-c45d966681d4/table/table.go:244
github.com/dgraph-io/badger/v2/table.BlockEvictHandler
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200615081930-c45d966681d4/table/table.go:190
github.com/dgraph-io/badger/v2.Open.func5
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/badger/v2@v2.0.1-rc1.0.20200615081930-c45d966681d4/db.go:310
github.com/dgraph-io/ristretto.(*Cache).processItems
	/home/ubuntu/go/pkg/mod/github.com/dgraph-io/ristretto@v0.0.2/cache.go:315
runtime.goexit
	/usr/local/go/src/runtime/asm_amd64.s:1373

@jarifibrahim
Copy link
Contributor Author

The crash originates from cache eviction handler.

badger/table/table.go

Lines 187 to 192 in 158d927

// BlockEvictHandler is used to reuse the byte slice stored in the block on cache eviction.
func BlockEvictHandler(value interface{}) {
if b, ok := value.(*block); ok {
b.decrRef()
}
}

Maybe the iterator is closing the same block twice somehow. Once in the setBlock

badger/table/iterator.go

Lines 43 to 46 in 158d927

func (itr *blockIterator) setBlock(b *block) {
// Decrement the ref for the old block. If the old block was compressed, we
// might be able to reuse it.
itr.block.decrRef()

and the second time in the itr.Close

badger/table/iterator.go

Lines 111 to 113 in 158d927

func (itr *blockIterator) Close() {
itr.block.decrRef()
}

I don't know how that could happen but that's the only way b.decrRef() can be called more than b.incrRef() and so the crash.

@damz
Copy link
Contributor

damz commented Jun 28, 2020

Looking at the code, I think there are at least two issues:

Issue #1: It seems to me that ristretto doesn't currently guarantee that the block has been added to the cache if and only if Set() returns true.

More specifically, in the case of updates, the item is first updated in the cache, then it is put on the setBuf channel. If that last operation failed (because under load the channel might get is full), Set() will return false, but the item has been put on the cache.

From there, Table.block() will decrease the reference count of the block, and return it with a refcount of 1, despite the block actually being in the cache and up for grab by other callers:

badger/table/table.go

Lines 588 to 591 in 158d927

// Decrement the block ref if we could not insert it in the cache.
if !t.opt.Cache.Set(key, blk, blk.size()) {
blk.decrRef()
}

The scenario where it can happen is a race between two goroutines loading the same block that is not yet in the cache:

  • Goroutine 1: does a miss on Get(), loads the block, decrypts it and uncompresses it, saves it in the cache, returns it with a refcount of 2
  • Goroutine 2: does a miss on Get(), loads the block, decrypts it and uncompresses it, tries to save it in the cache, but it is already there, ristretto swaps the item in the cache for the new block, but it happens to be under load and cannot put the item on the setBuf channel, returns false, the block has been put in the cache, but it is returned with a refcount of 1

Issue #2: It seems like the current code assumes that the evict handler is always called when an item is removed from the cache. But that does not seem to be true when items are updated. The evict handler will not be called on the removed version. As a consequence, blocks will leak when two goroutines race, like above, to load an item that is not in the cache. Luckily, the only effect of this is more memory garbage.

@damz
Copy link
Contributor

damz commented Jun 28, 2020

because under load the channel might get is full

I wondered if that was plausible or not, and it is:

  • (*Table).DecrRef() issues a cache deletion for all the block cache keys of the table in a tight loop:

badger/table/table.go

Lines 176 to 179 in e013bfd

// Delete all blocks from the cache.
for i := 0; i < t.noOfBlocks; i++ {
t.opt.Cache.Del(t.blockCacheKey(i))
}

  • and (*levelHandler).replaceTables() issues a (*Table).DecrRef() in a tight loop for all the deleted tables after a compaction:

return decrRefs(toDel)

badger/level_handler.go

Lines 168 to 175 in e013bfd

func decrRefs(tables []*table.Table) error {
for _, table := range tables {
if err := table.DecrRef(); err != nil {
return err
}
}
return nil
}

At the default table (64 MB) and block size (4 kB), there are 16384 blocks per table. At the default level multiplier, you expect a typical compaction to delete 11 tables (10 at the bottom layer, one at the top layer), so each compaction generates 180224 cache deletion requests in a relatively tight loop (but only relatively, there are a bunch of syscalls in (*Table).DecrRef()). In addition, there can be multiple compactions cleaning up tables at the same time.

The setBuf channel in ristretto has 32768 entries, so it sounds highly likely to me a real-life application could end up filling the set buffer, triggering the issue.

@jarifibrahim
Copy link
Contributor Author

This is fixed in ristretto via dgraph-io/ristretto#167 and ristretto was updated in badger via #1391 .

Thank you for the help @damz !

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/crash This issue causes a panic or some other of exception that causes a crash. help wanted Please help us fix this! kind/bug Something is broken. priority/P1 Serious issue that requires eventual attention (can wait a bit) status/accepted We accept to investigate or work on it.
Development

No branches or pull requests

3 participants