From b5ee7af0e9a6e17a20a25a8b8b0dd842d96ab846 Mon Sep 17 00:00:00 2001
From: Manish R Jain <manish@dgraph.io>
Date: Tue, 17 Jul 2018 17:37:39 -0700
Subject: [PATCH] Simplify txn commits and aborts (#2491)

Txn:

- Commits have been separated into CommitToMemory and CommitToDisk. This logic is a lot simpler and clearer than before. In particular, CommitToDisk does not try to generate any state. It just serializes the posting list stored at startTs and pushes that to Badger.
- Multiple txns doing CommitToDisk can now be batched up into fewer transactions and use callbacks to execute quickly. Added a helper struct for that.
- Remove the concept of activeTxns in posting/list.go. It's a repetition.
- Simplify txn.deltas, and removed as much as I could.
- Removed abortMutation methods from txn (mvcc.go) and posting/list.go

Zero:

- TryAbort returns OracleDelta, which can be directly proposed in the group.
- abortOldTransactions uses this to ensure that in case we missed a transaction update, the result of the abort can then be proposed to the group, so group snapshot doesn't get stuck.
---
 dgraph/cmd/server/http_test.go |   8 +-
 dgraph/cmd/zero/oracle.go      |  15 +-
 posting/index.go               |  16 +-
 posting/index_test.go          |   8 +-
 posting/list.go                |  95 ++++----
 posting/list_test.go           |  53 +++--
 posting/mvcc.go                | 160 ++++----------
 posting/oracle.go              |  19 +-
 protos/intern/internal.pb.go   | 387 ++++++++++++++++-----------------
 protos/internal.proto          |   2 +-
 query/common_test.go           |   2 +-
 worker/draft.go                |  84 ++++---
 worker/mutation.go             |  36 +--
 worker/predicate_test.go       |   6 +-
 x/badger.go                    |  48 ++++
 15 files changed, 457 insertions(+), 482 deletions(-)
 create mode 100644 x/badger.go

diff --git a/dgraph/cmd/server/http_test.go b/dgraph/cmd/server/http_test.go
index 701a39e3..f3602731 100644
--- a/dgraph/cmd/server/http_test.go
+++ b/dgraph/cmd/server/http_test.go
@@ -9,6 +9,7 @@ package server
 
 import (
 	"bytes"
+	"encoding/base64"
 	"encoding/json"
 	"errors"
 	"fmt"
@@ -160,7 +161,12 @@ func TestTransactionBasic(t *testing.T) {
 	require.NoError(t, err)
 	require.Equal(t, mts, ts)
 	sort.Strings(keys)
-	require.Equal(t, 4, len(keys))
+	for _, k := range keys {
+		kb, err := base64.StdEncoding.DecodeString(k)
+		require.NoError(t, err)
+		_ = x.Parse(kb) // Ensure that we can parse the key.
+	}
+	require.Equal(t, 3, len(keys))
 
 	data, _, err := queryWithTs(q1, 0)
 	require.NoError(t, err)
diff --git a/dgraph/cmd/zero/oracle.go b/dgraph/cmd/zero/oracle.go
index f91568c4..8a980f27 100644
--- a/dgraph/cmd/zero/oracle.go
+++ b/dgraph/cmd/zero/oracle.go
@@ -268,6 +268,8 @@ func (s *Server) proposeTxn(ctx context.Context, src *api.TxnContext) error {
 	return s.Node.proposeAndWait(ctx, &zp)
 }
 
+// TODO: This function implies that src has been modified to represent the latest status of the
+// transaction. I don't think that's happening properly.
 func (s *Server) commit(ctx context.Context, src *api.TxnContext) error {
 	if src.Aborted {
 		return s.proposeTxn(ctx, src)
@@ -435,18 +437,21 @@ OUTER:
 	}
 }
 
-func (s *Server) TryAbort(ctx context.Context, txns *intern.TxnTimestamps) (*intern.TxnTimestamps, error) {
-	commitTimestamps := new(intern.TxnTimestamps)
+func (s *Server) TryAbort(ctx context.Context,
+	txns *intern.TxnTimestamps) (*intern.OracleDelta, error) {
+	delta := &intern.OracleDelta{}
 	for _, startTs := range txns.Ts {
 		// Do via proposals to avoid race
 		tctx := &api.TxnContext{StartTs: startTs, Aborted: true}
 		if err := s.proposeTxn(ctx, tctx); err != nil {
-			return commitTimestamps, err
+			return delta, err
 		}
 		// Txn should be aborted if not already committed.
-		commitTimestamps.Ts = append(commitTimestamps.Ts, s.orc.commitTs(startTs))
+		delta.Txns = append(delta.Txns, &intern.TxnStatus{
+			StartTs:  startTs,
+			CommitTs: s.orc.commitTs(startTs)})
 	}
-	return commitTimestamps, nil
+	return delta, nil
 }
 
 // Timestamps is used to assign startTs for a new transaction
diff --git a/posting/index.go b/posting/index.go
index 99e5031e..0c6131fc 100644
--- a/posting/index.go
+++ b/posting/index.go
@@ -525,10 +525,10 @@ func rebuildCountIndex(ctx context.Context, attr string, reverse bool, errCh cha
 					err = txn.addCountMutation(ctx, t, uint32(len), reverse)
 				}
 				if err == nil {
-					err = txn.CommitMutationsMemory(ctx, txn.StartTs)
+					err = txn.CommitToMemory(txn.StartTs)
 				}
 				if err != nil {
-					txn.AbortMutations(ctx)
+					txn.CommitToMemory(0)
 				}
 				txn.deltas = nil
 			}
@@ -654,9 +654,9 @@ func RebuildReverseEdges(ctx context.Context, attr string, startTs uint64) error
 			txn := &Txn{StartTs: startTs}
 			for it := range ch {
 				addReversePostings(it.uid, it.list, txn)
-				err = txn.CommitMutationsMemory(ctx, txn.StartTs)
+				err = txn.CommitToMemory(txn.StartTs)
 				if err != nil {
-					txn.AbortMutations(ctx)
+					txn.CommitToMemory(0)
 				}
 				txn.deltas = nil
 			}
@@ -780,9 +780,9 @@ func RebuildListType(ctx context.Context, attr string, startTs uint64) error {
 					return
 				}
 
-				err = txn.CommitMutationsMemory(ctx, txn.StartTs)
+				err = txn.CommitToMemory(txn.StartTs)
 				if err != nil {
-					txn.AbortMutations(ctx)
+					txn.CommitToMemory(0)
 				}
 				txn.deltas = nil
 			}
@@ -862,9 +862,9 @@ func RebuildIndex(ctx context.Context, attr string, startTs uint64) error {
 			txn := &Txn{StartTs: startTs}
 			for it := range ch {
 				addPostingsToIndex(it.uid, it.list, txn)
-				err = txn.CommitMutationsMemory(ctx, txn.StartTs)
+				err = txn.CommitToMemory(txn.StartTs)
 				if err != nil {
-					txn.AbortMutations(ctx)
+					txn.CommitToMemory(0)
 				}
 				txn.deltas = nil
 			}
diff --git a/posting/index_test.go b/posting/index_test.go
index a1fa86a1..11e003eb 100644
--- a/posting/index_test.go
+++ b/posting/index_test.go
@@ -116,7 +116,11 @@ func addMutation(t *testing.T, l *List, edge *intern.DirectedEdge, op uint32,
 		err := l.AddMutation(context.Background(), txn, edge)
 		require.NoError(t, err)
 	}
-	require.NoError(t, txn.CommitMutations(context.Background(), commitTs))
+
+	writer := &x.TxnWriter{DB: pstore}
+	require.NoError(t, txn.CommitToDisk(writer, commitTs))
+	require.NoError(t, writer.Flush())
+	require.NoError(t, txn.CommitToMemory(commitTs))
 }
 
 const schemaVal = `
@@ -225,7 +229,7 @@ func addReverseEdge(t *testing.T, attr string, src uint64,
 		StartTs: startTs,
 	}
 	txn.addReverseMutation(context.Background(), edge)
-	require.NoError(t, txn.CommitMutations(context.Background(), commitTs))
+	require.NoError(t, txn.CommitToMemory(commitTs))
 }
 
 func TestRebuildIndex(t *testing.T) {
diff --git a/posting/list.go b/posting/list.go
index 24e202e1..f53e8bb7 100644
--- a/posting/list.go
+++ b/posting/list.go
@@ -65,8 +65,7 @@ type List struct {
 	mutationMap   map[uint64]*intern.PostingList
 	minTs         uint64 // commit timestamp of immutable layer, reject reads before this ts.
 	commitTs      uint64 // last commitTs of this pl
-	activeTxns    map[uint64]struct{}
-	deleteMe      int32 // Using atomic for this, to avoid expensive SetForDeletion operation.
+	deleteMe      int32  // Using atomic for this, to avoid expensive SetForDeletion operation.
 	estimatedSize int32
 	numCommits    int
 	onDisk        int32 // Using atomic, Was written to disk atleast once.
@@ -225,8 +224,10 @@ func (l *List) EstimatedSize() int32 {
 func (l *List) SetForDeletion() bool {
 	l.Lock()
 	defer l.Unlock()
-	if len(l.activeTxns) > 0 {
-		return false
+	for _, plist := range l.mutationMap {
+		if plist.Commit == 0 {
+			return false
+		}
 	}
 	atomic.StoreInt32(&l.deleteMe, 1)
 	return true
@@ -347,71 +348,57 @@ func (l *List) addMutation(ctx context.Context, txn *Txn, t *intern.DirectedEdge
 			tr.LazyPrintf("updated mutation layer %v %v %v", dur, len(l.mutationMap), len(l.plist.Uids))
 		}
 	}
-	l.activeTxns[txn.StartTs] = struct{}{}
-	txn.AddDelta(l.key, mpost, checkConflict)
-	return nil
-}
-
-func (l *List) AbortTransaction(ctx context.Context, startTs uint64) error {
-	l.Lock()
-	defer l.Unlock()
-	return l.abortTransaction(ctx, startTs)
-}
-
-func (l *List) abortTransaction(ctx context.Context, startTs uint64) error {
-	if atomic.LoadInt32(&l.deleteMe) == 1 {
-		if tr, ok := trace.FromContext(ctx); ok {
-			tr.LazyPrintf("DELETEME set to true. Temporary error.")
-		}
-		return ErrRetry
-	}
-	l.AssertLock()
-	if plist, ok := l.mutationMap[startTs]; ok {
-		atomic.AddInt32(&l.estimatedSize, -1*int32(plist.Size()))
-	}
-	delete(l.mutationMap, startTs)
-	delete(l.activeTxns, startTs)
+	txn.AddDelta(l.key, checkConflict)
 	return nil
 }
 
-func (l *List) AlreadyCommitted(startTs uint64) bool {
+// GetMutation returns a marshaled version of posting list mutation stored internally.
+func (l *List) GetMutation(startTs uint64) []byte {
 	l.RLock()
 	defer l.RUnlock()
-	_, ok := l.activeTxns[startTs]
-	return !ok
+	if pl, ok := l.mutationMap[startTs]; ok {
+		data, err := pl.Marshal()
+		x.Check(err)
+		return data
+	}
+	return nil
 }
 
-func (l *List) CommitMutation(ctx context.Context, startTs, commitTs uint64) error {
+func (l *List) CommitMutation(startTs, commitTs uint64) error {
 	l.Lock()
 	defer l.Unlock()
-	return l.commitMutation(ctx, startTs, commitTs)
+	return l.commitMutation(startTs, commitTs)
 }
 
-func (l *List) commitMutation(ctx context.Context, startTs, commitTs uint64) error {
+func (l *List) commitMutation(startTs, commitTs uint64) error {
 	if atomic.LoadInt32(&l.deleteMe) == 1 {
-		if tr, ok := trace.FromContext(ctx); ok {
-			tr.LazyPrintf("DELETEME set to true. Temporary error.")
-		}
 		return ErrRetry
 	}
 
 	l.AssertLock()
-	if plist, ok := l.mutationMap[startTs]; !ok {
+	plist, ok := l.mutationMap[startTs]
+	if !ok {
 		// It was already committed, might be happening due to replay.
 		return nil
-	} else {
-		plist.Commit = commitTs
-		for _, mpost := range plist.Postings {
-			mpost.CommitTs = commitTs
-		}
-		l.numCommits += len(plist.Postings)
+	}
+	if commitTs == 0 {
+		// Abort mutation.
+		atomic.AddInt32(&l.estimatedSize, -1*int32(plist.Size()))
+		delete(l.mutationMap, startTs)
+		return nil
 	}
 
+	// We have a valid commit.
+	plist.Commit = commitTs
+	for _, mpost := range plist.Postings {
+		mpost.CommitTs = commitTs
+	}
+	l.numCommits += len(plist.Postings)
+
 	if commitTs > l.commitTs {
 		// This is for rolling up the posting list.
 		l.commitTs = commitTs
 	}
-	delete(l.activeTxns, startTs)
 
 	// Calculate 5% of immutable layer
 	numUids := (bp128.NumIntegers(l.plist.Uids) * 5) / 100
@@ -445,7 +432,10 @@ func (l *List) Conflicts(readTs uint64) []uint64 {
 	l.RLock()
 	defer l.RUnlock()
 	var conflicts []uint64
-	for ts := range l.activeTxns {
+	for ts, pl := range l.mutationMap {
+		if pl.Commit > 0 {
+			continue
+		}
 		if ts < readTs {
 			conflicts = append(conflicts, ts)
 		}
@@ -621,7 +611,6 @@ func doAsyncWrite(commitTs uint64, key []byte, data []byte, meta byte, f func(er
 func (l *List) MarshalToKv() (*intern.KV, error) {
 	l.Lock()
 	defer l.Unlock()
-	x.AssertTrue(len(l.activeTxns) == 0)
 	if err := l.rollup(); err != nil {
 		return nil, err
 	}
@@ -714,6 +703,16 @@ func (l *List) SyncIfDirty(delFromCache bool) (committed bool, err error) {
 	return l.syncIfDirty(delFromCache)
 }
 
+func (l *List) hasPendingTxn() bool {
+	l.AssertRLock()
+	for _, pl := range l.mutationMap {
+		if pl.Commit == 0 {
+			return true
+		}
+	}
+	return false
+}
+
 // Merge mutation layer and immutable layer.
 func (l *List) syncIfDirty(delFromCache bool) (committed bool, err error) {
 	// We no longer set posting list to empty.
@@ -722,7 +721,7 @@ func (l *List) syncIfDirty(delFromCache bool) (committed bool, err error) {
 	}
 	if delFromCache {
 		// Don't evict if there is pending transaction.
-		x.AssertTrue(len(l.activeTxns) == 0)
+		x.AssertTrue(!l.hasPendingTxn())
 	}
 
 	lmlayer := len(l.mutationMap)
diff --git a/posting/list_test.go b/posting/list_test.go
index ebecb2c4..0c333b7f 100644
--- a/posting/list_test.go
+++ b/posting/list_test.go
@@ -101,7 +101,7 @@ func TestAddMutation(t *testing.T) {
 	edge.ValueId = 9
 	edge.Label = "anti-testing"
 	addMutationHelper(t, l, edge, Set, txn)
-	l.CommitMutation(context.Background(), 1, 2)
+	l.CommitMutation(1, 2)
 
 	uids := []uint64{9, 69, 81}
 	checkUids(t, l, uids, 3)
@@ -144,7 +144,7 @@ func TestAddMutation_Value(t *testing.T) {
 	checkValue(t, ol, "oh hey there", txn.StartTs)
 
 	// Run the same check after committing.
-	ol.CommitMutation(context.Background(), txn.StartTs, txn.StartTs+1)
+	ol.CommitMutation(txn.StartTs, txn.StartTs+1)
 	_, err = ol.SyncIfDirty(false)
 	require.NoError(t, err)
 	checkValue(t, ol, "oh hey there", uint64(3))
@@ -168,7 +168,7 @@ func TestAddMutation_jchiu1(t *testing.T) {
 	}
 	txn := &Txn{StartTs: 1}
 	addMutationHelper(t, ol, edge, Set, txn)
-	ol.CommitMutation(context.Background(), 1, uint64(2))
+	ol.CommitMutation(1, uint64(2))
 	merged, err := ol.SyncIfDirty(false)
 	require.NoError(t, err)
 	require.True(t, merged)
@@ -227,7 +227,7 @@ func TestAddMutation_DelSet(t *testing.T) {
 	}
 	txn = &Txn{StartTs: 2}
 	addMutationHelper(t, ol, edge, Set, txn)
-	ol.CommitMutation(context.Background(), 2, uint64(3))
+	ol.CommitMutation(2, uint64(3))
 	require.EqualValues(t, 1, ol.Length(3, 0))
 	checkValue(t, ol, "newcars", 3)
 }
@@ -242,7 +242,7 @@ func TestAddMutation_DelRead(t *testing.T) {
 	}
 	txn := &Txn{StartTs: 1}
 	addMutationHelper(t, ol, edge, Set, txn)
-	ol.CommitMutation(context.Background(), 1, uint64(2))
+	ol.CommitMutation(1, uint64(2))
 	require.EqualValues(t, 1, ol.Length(2, 0))
 	checkValue(t, ol, "newcars", 2)
 
@@ -261,7 +261,7 @@ func TestAddMutation_DelRead(t *testing.T) {
 	require.EqualValues(t, 0, ol.Length(3, 0))
 
 	// Commit sp* only in oracle, don't apply to pl yet
-	ol.CommitMutation(context.Background(), 3, 5)
+	ol.CommitMutation(3, 5)
 
 	// This read should ignore sp*, since readts is 4 and it was committed at 5
 	require.EqualValues(t, 1, ol.Length(4, 0))
@@ -306,7 +306,7 @@ func TestAddMutation_jchiu2_Commit(t *testing.T) {
 	}
 	txn := &Txn{StartTs: 1}
 	addMutationHelper(t, ol, edge, Del, txn)
-	ol.CommitMutation(context.Background(), 1, uint64(2))
+	ol.CommitMutation(1, uint64(2))
 	require.EqualValues(t, 0, ol.Length(uint64(3), 0))
 
 	// Set value to newcars, but don't merge yet.
@@ -316,7 +316,7 @@ func TestAddMutation_jchiu2_Commit(t *testing.T) {
 	}
 	txn = &Txn{StartTs: 3}
 	addMutationHelper(t, ol, edge, Set, txn)
-	ol.CommitMutation(context.Background(), 3, uint64(4))
+	ol.CommitMutation(3, uint64(4))
 	require.EqualValues(t, 1, ol.Length(5, 0))
 	checkValue(t, ol, "newcars", 5)
 }
@@ -333,7 +333,7 @@ func TestAddMutation_jchiu3(t *testing.T) {
 	}
 	txn := &Txn{StartTs: 1}
 	addMutationHelper(t, ol, edge, Set, txn)
-	ol.CommitMutation(context.Background(), 1, uint64(2))
+	ol.CommitMutation(1, uint64(2))
 	require.Equal(t, 1, ol.Length(uint64(3), 0))
 	merged, err := ol.SyncIfDirty(false)
 	require.NoError(t, err)
@@ -380,7 +380,7 @@ func TestAddMutation_mrjn1(t *testing.T) {
 	}
 	txn := &Txn{StartTs: 1}
 	addMutationHelper(t, ol, edge, Set, txn)
-	ol.CommitMutation(context.Background(), 1, uint64(2))
+	ol.CommitMutation(1, uint64(2))
 	merged, err := ol.SyncIfDirty(false)
 	require.NoError(t, err)
 	require.True(t, merged)
@@ -422,7 +422,6 @@ func TestAddMutation_mrjn1(t *testing.T) {
 }
 
 func TestMillion(t *testing.T) {
-	ctx := context.Background()
 	key := x.DataKey("bal", 1331)
 	ol, err := getNew(key, ps)
 	require.NoError(t, err)
@@ -434,7 +433,7 @@ func TestMillion(t *testing.T) {
 		}
 		txn := Txn{StartTs: uint64(i)}
 		addMutationHelper(t, ol, edge, Set, &txn)
-		require.NoError(t, ol.CommitMutation(ctx, uint64(i), uint64(i)+1))
+		require.NoError(t, ol.CommitMutation(uint64(i), uint64(i)+1))
 		commits++
 	}
 	opt := ListOptions{ReadTs: uint64(N) + 1}
@@ -470,10 +469,10 @@ func TestAddMutation_mrjn2(t *testing.T) {
 		require.EqualValues(t, uint64(i), list.Uids[0])
 	}
 	require.EqualValues(t, 0, ol.Length(readTs, 0))
-	require.NoError(t, ol.AbortTransaction(ctx, uint64(1)))
-	require.NoError(t, ol.CommitMutation(ctx, 3, 4))
-	require.NoError(t, ol.CommitMutation(ctx, 6, 10))
-	require.NoError(t, ol.CommitMutation(ctx, 9, 14))
+	require.NoError(t, ol.CommitMutation(1, 0))
+	require.NoError(t, ol.CommitMutation(3, 4))
+	require.NoError(t, ol.CommitMutation(6, 10))
+	require.NoError(t, ol.CommitMutation(9, 14))
 	require.EqualValues(t, 3, ol.Length(15, 0)) // The three commits.
 
 	{
@@ -494,7 +493,7 @@ func TestAddMutation_mrjn2(t *testing.T) {
 		require.NoError(t, err)
 
 		require.EqualValues(t, 3, ol.Length(15, 0)) // The three commits should still be found.
-		require.NoError(t, ol.CommitMutation(ctx, 7, 11))
+		require.NoError(t, ol.CommitMutation(7, 11))
 
 		require.EqualValues(t, 2, ol.Length(10, 0)) // Two commits should be found.
 		require.EqualValues(t, 1, ol.Length(12, 0)) // Only one commit should be found.
@@ -508,12 +507,12 @@ func TestAddMutation_mrjn2(t *testing.T) {
 		txn := &Txn{StartTs: 5}
 		err := ol.AddMutation(ctx, txn, edge)
 		require.NoError(t, err)
-		require.NoError(t, ol.CommitMutation(ctx, 5, 7))
+		require.NoError(t, ol.CommitMutation(5, 7))
 
 		// Commits are:
 		// 4, 7 (Delete *), 10, 11 (Delete *), 14
 		require.EqualValues(t, 1, ol.Length(8, 0)) // Nothing below 8, but consider itself.
-		require.NoError(t, ol.AbortTransaction(ctx, 8))
+		require.NoError(t, ol.CommitMutation(8, 0))
 		require.EqualValues(t, 0, ol.Length(8, 0))  // Nothing <= 8.
 		require.EqualValues(t, 1, ol.Length(10, 0)) // Find committed 10.
 		require.EqualValues(t, 1, ol.Length(12, 0)) // Find committed 11.
@@ -544,7 +543,7 @@ func TestAddMutation_gru(t *testing.T) {
 			Label:   "gru",
 		}
 		addMutationHelper(t, ol, edge, Set, txn)
-		ol.CommitMutation(context.Background(), 1, uint64(2))
+		ol.CommitMutation(1, uint64(2))
 		merged, err := ol.SyncIfDirty(false)
 		require.NoError(t, err)
 		require.True(t, merged)
@@ -562,7 +561,7 @@ func TestAddMutation_gru(t *testing.T) {
 			Label:   "gru",
 		}
 		addMutationHelper(t, ol, edge, Del, txn)
-		ol.CommitMutation(context.Background(), 3, uint64(4))
+		ol.CommitMutation(3, uint64(4))
 		merged, err := ol.SyncIfDirty(false)
 		require.NoError(t, err)
 		require.True(t, merged)
@@ -588,7 +587,7 @@ func TestAddMutation_gru2(t *testing.T) {
 		}
 		txn = &Txn{StartTs: 1}
 		addMutationHelper(t, ol, edge, Set, txn)
-		ol.CommitMutation(context.Background(), 1, uint64(2))
+		ol.CommitMutation(1, uint64(2))
 		merged, err := ol.SyncIfDirty(false)
 		require.NoError(t, err)
 		require.True(t, merged)
@@ -614,7 +613,7 @@ func TestAddMutation_gru2(t *testing.T) {
 		}
 		addMutationHelper(t, ol, edge, Set, txn)
 
-		ol.CommitMutation(context.Background(), 3, uint64(4))
+		ol.CommitMutation(3, uint64(4))
 	}
 
 	// Posting list should just have the new tag.
@@ -637,7 +636,7 @@ func TestAddAndDelMutation(t *testing.T) {
 		}
 		txn := &Txn{StartTs: 1}
 		addMutationHelper(t, ol, edge, Set, txn)
-		ol.CommitMutation(context.Background(), 1, uint64(2))
+		ol.CommitMutation(1, uint64(2))
 		merged, err := ol.SyncIfDirty(false)
 		require.NoError(t, err)
 		require.True(t, merged)
@@ -652,7 +651,7 @@ func TestAddAndDelMutation(t *testing.T) {
 		txn := &Txn{StartTs: 3}
 		addMutationHelper(t, ol, edge, Del, txn)
 		addMutationHelper(t, ol, edge, Del, txn)
-		ol.CommitMutation(context.Background(), 3, uint64(4))
+		ol.CommitMutation(3, uint64(4))
 
 		checkUids(t, ol, []uint64{}, 5)
 
@@ -786,7 +785,7 @@ func TestDelete(t *testing.T) {
 	edge.Value = []byte(x.Star)
 	addMutationHelper(t, ol, edge, Del, txn)
 	require.EqualValues(t, 0, ol.Length(txn.StartTs, 0))
-	ol.CommitMutation(context.Background(), txn.StartTs, txn.StartTs+1)
+	ol.CommitMutation(txn.StartTs, txn.StartTs+1)
 	commited, err := ol.SyncIfDirty(false)
 	require.NoError(t, err)
 	require.True(t, commited)
@@ -814,7 +813,7 @@ func TestAfterUIDCountWithCommit(t *testing.T) {
 	require.EqualValues(t, 0, ol.Length(txn.StartTs, 400))
 
 	// Commit to database.
-	ol.CommitMutation(context.Background(), txn.StartTs, txn.StartTs+1)
+	ol.CommitMutation(txn.StartTs, txn.StartTs+1)
 	merged, err := ol.SyncIfDirty(false)
 	require.NoError(t, err)
 	require.True(t, merged)
diff --git a/posting/mvcc.go b/posting/mvcc.go
index 906a2b71..9544c3da 100644
--- a/posting/mvcc.go
+++ b/posting/mvcc.go
@@ -9,10 +9,8 @@ package posting
 
 import (
 	"bytes"
-	"context"
 	"encoding/base64"
 	"math"
-	"sort"
 	"sync/atomic"
 	"time"
 
@@ -37,171 +35,89 @@ func (t *Txn) ShouldAbort() bool {
 	return atomic.LoadUint32(&t.shouldAbort) > 0
 }
 
-func (t *Txn) AddDelta(key []byte, p *intern.Posting, checkConflict bool) {
+func (t *Txn) AddDelta(key []byte, checkConflict bool) {
 	t.Lock()
 	defer t.Unlock()
-	t.deltas = append(t.deltas, delta{key: key, posting: p, checkConflict: checkConflict})
+	if t.deltas == nil {
+		t.deltas = make(map[string]bool)
+	}
+	t.deltas[string(key)] = checkConflict
 }
 
 func (t *Txn) Fill(ctx *api.TxnContext) {
 	t.Lock()
 	defer t.Unlock()
 	ctx.StartTs = t.StartTs
-	for i := t.nextKeyIdx; i < len(t.deltas); i++ {
-		d := t.deltas[i]
-		if d.checkConflict {
+	for key, checkConflict := range t.deltas {
+		if checkConflict {
 			// Instead of taking a fingerprint of the keys, send the whole key to Zero. So, Zero can
 			// parse the key and check if that predicate is undergoing a move, hence avoiding #2338.
-			k := base64.StdEncoding.EncodeToString(d.key)
+			k := base64.StdEncoding.EncodeToString([]byte(key))
 			ctx.Keys = append(ctx.Keys, k)
 		}
 	}
-	t.nextKeyIdx = len(t.deltas)
 }
 
 // Don't call this for schema mutations. Directly commit them.
+// This function only stores deltas to the commit timestamps. It does not try to generate a state.
 // TODO: Simplify this function. All it should be doing is to store the deltas, and not try to
 // generate state. The state should only be generated by rollup, which in turn should look at the
 // last Snapshot Ts, to determine how much of the PL to rollup. We only want to roll up the deltas,
 // with commit ts <= snapshot ts, and not above.
-func (tx *Txn) CommitMutations(ctx context.Context, commitTs uint64) error {
+func (tx *Txn) CommitToDisk(writer *x.TxnWriter, commitTs uint64) error {
+	if commitTs == 0 {
+		return nil
+	}
 	tx.Lock()
 	defer tx.Unlock()
 
-	txn := pstore.NewTransactionAt(commitTs, true)
-	defer txn.Discard()
-	// Sort by keys so that we have all postings for same pl side by side.
-	sort.SliceStable(tx.deltas, func(i, j int) bool {
-		return bytes.Compare(tx.deltas[i].key, tx.deltas[j].key) < 0
-	})
 	// TODO: Simplify this. All we need to do is to get the PL for the key, and if it has the
 	// postings for the startTs, we commit them. Otherwise, we skip.
 	// Also, if the snapshot read ts is above the commit ts, then we just delete the postings from
 	// memory, instead of writing them back again.
-	var prevKey []byte
-	var pl *intern.PostingList
-	var plist *List
-	var err error
-	i := 0
-	for i < len(tx.deltas) {
-		d := tx.deltas[i]
-		if !bytes.Equal(prevKey, d.key) {
-			plist, err = Get(d.key)
-			if err != nil {
-				return err
-			}
-			if plist.AlreadyCommitted(tx.StartTs) {
-				// Delta already exists, so skip the key
-				// There won't be any race from lru eviction, because we don't
-				// commit in memory unless we write delta to disk.
-				i++
-				for i < len(tx.deltas) && bytes.Equal(tx.deltas[i].key, d.key) {
-					i++
-				}
-				continue
-			}
-			pl = new(intern.PostingList)
-		}
-		prevKey = d.key
-		var meta byte
-		if d.posting.Op == Del && bytes.Equal(d.posting.Value, []byte(x.Star)) {
-			pl.Postings = pl.Postings[:0]
-			// Indicates that this is the full posting list.
-			meta = BitEmptyPosting
-		} else {
-			midx := sort.Search(len(pl.Postings), func(idx int) bool {
-				mp := pl.Postings[idx]
-				return d.posting.Uid <= mp.Uid
-			})
-			if midx >= len(pl.Postings) {
-				pl.Postings = append(pl.Postings, d.posting)
-			} else if pl.Postings[midx].Uid == d.posting.Uid {
-				// Replace
-				pl.Postings[midx] = d.posting
-			} else {
-				pl.Postings = append(pl.Postings, nil)
-				copy(pl.Postings[midx+1:], pl.Postings[midx:])
-				pl.Postings[midx] = d.posting
-			}
-			meta = bitDeltaPosting
-		}
-
-		// delta postings are pointers to the postings present in the Pl present in lru.
-		// commitTs is accessed using RLock & atomics except in marshal so no RLock.
-		// TODO: Fix this hack later
-		plist.Lock()
-		val, err := pl.Marshal()
-		plist.Unlock()
-		x.Check(err)
-		if err = txn.SetWithMeta([]byte(d.key), val, meta); err == badger.ErrTxnTooBig {
-			if err := txn.CommitAt(commitTs, nil); err != nil {
-				return err
-			}
-			txn = pstore.NewTransactionAt(commitTs, true)
-			if err := txn.SetWithMeta([]byte(d.key), val, meta); err != nil {
-				return err
-			}
-		} else if err != nil {
-			return err
-		}
-		i++
-	}
-	if err := txn.CommitAt(commitTs, nil); err != nil {
-		return err
-	}
-	return tx.commitMutationsMemory(ctx, commitTs)
-}
-
-func (tx *Txn) CommitMutationsMemory(ctx context.Context, commitTs uint64) error {
-	tx.Lock()
-	defer tx.Unlock()
-	return tx.commitMutationsMemory(ctx, commitTs)
-}
 
-func (tx *Txn) commitMutationsMemory(ctx context.Context, commitTs uint64) error {
-	for _, d := range tx.deltas {
-		plist, err := Get(d.key)
+	for key := range tx.deltas {
+		plist, err := Get([]byte(key))
 		if err != nil {
 			return err
 		}
-		err = plist.CommitMutation(ctx, tx.StartTs, commitTs)
-		for err == ErrRetry {
-			time.Sleep(5 * time.Millisecond)
-			plist, err = Get(d.key)
-			if err != nil {
-				return err
-			}
-			err = plist.CommitMutation(ctx, tx.StartTs, commitTs)
+		data := plist.GetMutation(tx.StartTs)
+		if data == nil {
+			continue
 		}
-		if err != nil {
+		if err := writer.SetAt([]byte(key), data, bitDeltaPosting, commitTs); err != nil {
 			return err
 		}
 	}
 	return nil
 }
 
-func (tx *Txn) AbortMutations(ctx context.Context) error {
+func (tx *Txn) CommitToMemory(commitTs uint64) error {
 	tx.Lock()
 	defer tx.Unlock()
-	for _, d := range tx.deltas {
-		plist, err := Get([]byte(d.key))
-		if err != nil {
-			return err
-		}
-		err = plist.AbortTransaction(ctx, tx.StartTs)
-		for err == ErrRetry {
-			time.Sleep(5 * time.Millisecond)
-			plist, err = Get(d.key)
+	// TODO: Figure out what shouldAbort is for, and use it correctly. This should really be
+	// shouldDiscard.
+	// defer func() {
+	// 	atomic.StoreUint32(&tx.shouldAbort, 1)
+	// }()
+	for key := range tx.deltas {
+		for {
+			plist, err := Get([]byte(key))
 			if err != nil {
 				return err
 			}
-			err = plist.AbortTransaction(ctx, tx.StartTs)
-		}
-		if err != nil {
+			err = plist.CommitMutation(tx.StartTs, commitTs)
+			if err == ErrRetry {
+				time.Sleep(5 * time.Millisecond)
+				continue
+			}
+			if err == nil {
+				break
+			}
+			x.Errorf("While commiting to memory: %v\n", err)
 			return err
 		}
 	}
-	atomic.StoreUint32(&tx.shouldAbort, 1)
 	return nil
 }
 
@@ -234,7 +150,6 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) {
 	l := new(List)
 	l.key = key
 	l.mutationMap = make(map[uint64]*intern.PostingList)
-	l.activeTxns = make(map[uint64]struct{})
 	l.plist = new(intern.PostingList)
 
 	// Iterates from highest Ts to lowest Ts
@@ -289,7 +204,6 @@ func getNew(key []byte, pstore *badger.ManagedDB) (*List, error) {
 	l := new(List)
 	l.key = key
 	l.mutationMap = make(map[uint64]*intern.PostingList)
-	l.activeTxns = make(map[uint64]struct{})
 	l.plist = new(intern.PostingList)
 	txn := pstore.NewTransactionAt(math.MaxUint64, false)
 	defer txn.Discard()
diff --git a/posting/oracle.go b/posting/oracle.go
index ded97bdc..c685fe1d 100644
--- a/posting/oracle.go
+++ b/posting/oracle.go
@@ -28,15 +28,6 @@ func init() {
 	o.init()
 }
 
-// This structure is useful to keep track of which keys were updated, and whether they should be
-// used for conflict detection or not. When a txn is marked committed or aborted, this is what we
-// use to go fetch the posting lists and update the txn status in them.
-type delta struct {
-	key           []byte
-	posting       *intern.Posting
-	checkConflict bool // Check conflict detection.
-}
-
 type Txn struct {
 	StartTs uint64
 
@@ -44,8 +35,10 @@ type Txn struct {
 	shouldAbort uint32
 	// Fields which can changed after init
 	sync.Mutex
-	deltas     []delta
-	nextKeyIdx int
+	// Deltas keeps track of the posting list keys, and whether they should be considered for
+	// conflict detection or not. When a txn is marked committed or aborted, we use the keys stored
+	// here to determine which posting lists to get and update.
+	deltas map[string]bool
 
 	// Keeps track of last update wall clock. We use this fact later to
 	// determine unhealthy, stale txns.
@@ -191,8 +184,8 @@ func (o *oracle) GetTxn(startTs uint64) *Txn {
 func (t *Txn) matchesDelta(ok func(key []byte) bool) bool {
 	t.Lock()
 	defer t.Unlock()
-	for _, d := range t.deltas {
-		if ok(d.key) {
+	for key := range t.deltas {
+		if ok([]byte(key)) {
 			return true
 		}
 	}
diff --git a/protos/intern/internal.pb.go b/protos/intern/internal.pb.go
index 2b896b29..24847b4e 100644
--- a/protos/intern/internal.pb.go
+++ b/protos/intern/internal.pb.go
@@ -2169,7 +2169,7 @@ type ZeroClient interface {
 	AssignUids(ctx context.Context, in *Num, opts ...grpc.CallOption) (*api.AssignedIds, error)
 	Timestamps(ctx context.Context, in *Num, opts ...grpc.CallOption) (*api.AssignedIds, error)
 	CommitOrAbort(ctx context.Context, in *api.TxnContext, opts ...grpc.CallOption) (*api.TxnContext, error)
-	TryAbort(ctx context.Context, in *TxnTimestamps, opts ...grpc.CallOption) (*TxnTimestamps, error)
+	TryAbort(ctx context.Context, in *TxnTimestamps, opts ...grpc.CallOption) (*OracleDelta, error)
 }
 
 type zeroClient struct {
@@ -2288,8 +2288,8 @@ func (c *zeroClient) CommitOrAbort(ctx context.Context, in *api.TxnContext, opts
 	return out, nil
 }
 
-func (c *zeroClient) TryAbort(ctx context.Context, in *TxnTimestamps, opts ...grpc.CallOption) (*TxnTimestamps, error) {
-	out := new(TxnTimestamps)
+func (c *zeroClient) TryAbort(ctx context.Context, in *TxnTimestamps, opts ...grpc.CallOption) (*OracleDelta, error) {
+	out := new(OracleDelta)
 	err := grpc.Invoke(ctx, "/intern.Zero/TryAbort", in, out, c.cc, opts...)
 	if err != nil {
 		return nil, err
@@ -2307,7 +2307,7 @@ type ZeroServer interface {
 	AssignUids(context.Context, *Num) (*api.AssignedIds, error)
 	Timestamps(context.Context, *Num) (*api.AssignedIds, error)
 	CommitOrAbort(context.Context, *api.TxnContext) (*api.TxnContext, error)
-	TryAbort(context.Context, *TxnTimestamps) (*TxnTimestamps, error)
+	TryAbort(context.Context, *TxnTimestamps) (*OracleDelta, error)
 }
 
 func RegisterZeroServer(s *grpc.Server, srv ZeroServer) {
@@ -13107,197 +13107,196 @@ var (
 func init() { proto.RegisterFile("internal.proto", fileDescriptorInternal) }
 
 var fileDescriptorInternal = []byte{
-	// 3060 bytes of a gzipped FileDescriptorProto
+	// 3054 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x39, 0x4b, 0x6f, 0x23, 0xc7,
 	0xd1, 0x9a, 0x21, 0x39, 0x1c, 0x16, 0x49, 0x2d, 0xdd, 0x5e, 0xef, 0xd2, 0xb4, 0x3f, 0x59, 0x1e,
 	0xdb, 0xbb, 0xb2, 0xbd, 0x96, 0xbd, 0xf2, 0xfa, 0xf1, 0xf9, 0xfb, 0x1c, 0x40, 0x2b, 0x71, 0xd7,
-	0xf4, 0xea, 0xe5, 0x26, 0xb5, 0x8e, 0x7d, 0x08, 0xd1, 0xe2, 0xb4, 0xa4, 0x81, 0x86, 0x33, 0xe3,
-	0xe9, 0xa1, 0x40, 0xf9, 0x98, 0x5b, 0x90, 0x3f, 0x10, 0xe4, 0x18, 0x20, 0xb7, 0x00, 0x01, 0x72,
-	0xcc, 0x3d, 0x40, 0x02, 0x04, 0x48, 0x72, 0xce, 0x25, 0x71, 0x6e, 0x01, 0x72, 0xc8, 0x3f, 0x08,
-	0xba, 0xba, 0xe7, 0x41, 0x8a, 0x2b, 0x6f, 0x5e, 0x27, 0x4e, 0x55, 0x57, 0x75, 0x75, 0x3d, 0xba,
-	0xaa, 0xba, 0x08, 0xcb, 0x5e, 0x90, 0xf0, 0x38, 0x60, 0xfe, 0x7a, 0x14, 0x87, 0x49, 0x48, 0x2c,
-	0x05, 0x77, 0x6a, 0x2c, 0xf2, 0x14, 0xca, 0xe9, 0x40, 0x79, 0xc7, 0x13, 0x09, 0x21, 0x50, 0x9e,
-	0x78, 0xae, 0x68, 0x1b, 0xab, 0xa5, 0x35, 0x8b, 0xe2, 0xb7, 0xf3, 0x19, 0xd4, 0x06, 0x4c, 0x9c,
-	0x3d, 0x66, 0xfe, 0x84, 0x93, 0x16, 0x94, 0xce, 0x99, 0xdf, 0x36, 0x56, 0x8d, 0xb5, 0x06, 0x95,
-	0x9f, 0x64, 0x03, 0xec, 0x73, 0xe6, 0x0f, 0x93, 0x8b, 0x88, 0xb7, 0xcd, 0x55, 0x63, 0x6d, 0x79,
-	0xe3, 0xe6, 0xba, 0x12, 0xb0, 0x7e, 0x10, 0x8a, 0xc4, 0x0b, 0x4e, 0xd6, 0x1f, 0x33, 0x7f, 0x70,
-	0x11, 0x71, 0x5a, 0x3d, 0x57, 0x1f, 0xce, 0x3e, 0xd4, 0xfb, 0xf1, 0xe8, 0xc1, 0x24, 0x18, 0x25,
-	0x5e, 0x18, 0x48, 0xa9, 0x01, 0x1b, 0x73, 0xdc, 0xb5, 0x46, 0xf1, 0x5b, 0xe2, 0x58, 0x7c, 0x22,
-	0xda, 0xa5, 0xd5, 0x92, 0xc4, 0xc9, 0x6f, 0xd2, 0x86, 0xaa, 0x27, 0xb6, 0xc2, 0x49, 0x90, 0xb4,
-	0xcb, 0xab, 0xc6, 0x9a, 0x4d, 0x53, 0xd0, 0xf9, 0x59, 0x09, 0x2a, 0x9f, 0x4d, 0x78, 0x7c, 0x81,
-	0x7c, 0x49, 0x12, 0xa7, 0x7b, 0xc9, 0x6f, 0x72, 0x1d, 0x2a, 0x3e, 0x0b, 0x4e, 0x44, 0xdb, 0xc4,
-	0xcd, 0x14, 0x40, 0x5e, 0x80, 0x1a, 0x3b, 0x4e, 0x78, 0x3c, 0x9c, 0x78, 0x6e, 0xbb, 0xb4, 0x6a,
-	0xac, 0x59, 0xd4, 0x46, 0xc4, 0xa1, 0xe7, 0x92, 0xe7, 0xc1, 0x76, 0xc3, 0xe1, 0xa8, 0x28, 0xcb,
-	0x0d, 0x51, 0x16, 0xb9, 0x0d, 0xf6, 0xc4, 0x73, 0x87, 0xbe, 0x27, 0x92, 0x76, 0x65, 0xd5, 0x58,
-	0xab, 0x6f, 0x34, 0x52, 0x85, 0xa5, 0x0d, 0x69, 0x75, 0xe2, 0xb9, 0x68, 0xcc, 0x75, 0xb0, 0x45,
-	0x3c, 0x1a, 0x1e, 0x4f, 0x82, 0x51, 0xdb, 0x42, 0xc2, 0x67, 0x53, 0xc2, 0x82, 0xf6, 0xb4, 0x2a,
-	0x14, 0x20, 0xd5, 0x8b, 0xf9, 0x39, 0x8f, 0x05, 0x6f, 0x57, 0x95, 0x48, 0x0d, 0x92, 0x7b, 0x50,
-	0x3f, 0x66, 0x23, 0x9e, 0x0c, 0x23, 0x16, 0xb3, 0x71, 0xdb, 0x9e, 0xdd, 0xec, 0x81, 0x5c, 0x3a,
-	0x90, 0x2b, 0x82, 0xc2, 0x71, 0x06, 0x90, 0x0f, 0xa0, 0x89, 0x90, 0x18, 0x1e, 0x7b, 0x7e, 0xc2,
-	0xe3, 0x76, 0x0d, 0xf9, 0x48, 0xc6, 0x87, 0xd8, 0x41, 0xcc, 0x39, 0x6d, 0x28, 0x42, 0x85, 0x21,
-	0xff, 0x03, 0xc0, 0xa7, 0x11, 0x0b, 0xdc, 0x21, 0xf3, 0xfd, 0x36, 0xe0, 0x59, 0x6a, 0x0a, 0xb3,
-	0xe9, 0xfb, 0xe4, 0xa6, 0x3c, 0x27, 0x73, 0x87, 0x89, 0x68, 0x37, 0x57, 0x8d, 0xb5, 0x32, 0xb5,
-	0x24, 0x38, 0x10, 0xd2, 0x32, 0xbe, 0x17, 0x0c, 0x25, 0xd4, 0x5e, 0xd6, 0x96, 0x91, 0x31, 0xb6,
-	0xe3, 0x05, 0x94, 0x33, 0x97, 0x56, 0x7d, 0xf5, 0xe1, 0xbc, 0x0f, 0x35, 0x0c, 0x27, 0x34, 0xd3,
-	0xeb, 0x60, 0x9d, 0x4b, 0x40, 0x45, 0x5d, 0x7d, 0xe3, 0x99, 0xf4, 0x7c, 0x59, 0xd4, 0x51, 0x4d,
-	0xe0, 0xac, 0x80, 0xbd, 0xc3, 0x82, 0x93, 0x34, 0x54, 0xa5, 0x1f, 0x91, 0xa9, 0x46, 0xf1, 0xdb,
-	0xf9, 0x83, 0x09, 0x16, 0xe5, 0x62, 0xe2, 0x27, 0xe4, 0x4d, 0x00, 0xe9, 0xa5, 0x31, 0x4b, 0x62,
-	0x6f, 0xaa, 0x77, 0x9e, 0xf5, 0x53, 0x6d, 0xe2, 0xb9, 0xbb, 0xb8, 0x4c, 0xee, 0x41, 0x03, 0x25,
-	0xa4, 0xe4, 0xe6, 0xec, 0x41, 0xb2, 0xb3, 0xd2, 0x3a, 0x92, 0x69, 0xae, 0x1b, 0x60, 0x61, 0x80,
-	0xa8, 0x20, 0x6d, 0x52, 0x0d, 0x91, 0xd7, 0xf4, 0x8d, 0x13, 0x7c, 0x94, 0x0c, 0x5d, 0x2e, 0xd2,
-	0x08, 0x6a, 0x66, 0xd8, 0x6d, 0x2e, 0x12, 0xf2, 0x1e, 0x28, 0xab, 0xa7, 0x42, 0x2b, 0x28, 0x94,
-	0xcc, 0x78, 0x55, 0x28, 0xa9, 0x48, 0xa7, 0xa5, 0xde, 0x85, 0xba, 0xd4, 0x35, 0xe5, 0xb2, 0x90,
-	0xab, 0x95, 0x69, 0xa6, 0xcd, 0x43, 0x41, 0x12, 0x69, 0x16, 0x69, 0x2a, 0x19, 0xad, 0x2a, 0xaa,
-	0xf0, 0xfb, 0xe9, 0x7d, 0xd5, 0x85, 0xca, 0x7e, 0xec, 0xf2, 0x78, 0xe1, 0xcd, 0x22, 0x50, 0x76,
-	0xb9, 0x18, 0xe1, 0xc5, 0xb7, 0x29, 0x7e, 0xe7, 0xb7, 0xad, 0x54, 0xb8, 0x6d, 0xce, 0xef, 0x0c,
-	0xa8, 0xf7, 0xc3, 0x38, 0xd9, 0xe5, 0x42, 0xb0, 0x13, 0x4e, 0x5e, 0x81, 0x4a, 0x28, 0xb7, 0xd5,
-	0xae, 0x69, 0xa6, 0x0a, 0xa0, 0x2c, 0xaa, 0xd6, 0xe6, 0x9c, 0x68, 0x5e, 0xed, 0xc4, 0xeb, 0x50,
-	0x51, 0xf7, 0x55, 0xde, 0xe5, 0x0a, 0x55, 0x80, 0x74, 0x52, 0x78, 0x7c, 0x2c, 0xb8, 0x72, 0x42,
-	0x85, 0x6a, 0xe8, 0x3f, 0x10, 0xc4, 0x47, 0x00, 0x52, 0xa1, 0x7f, 0x25, 0xde, 0x9e, 0x5a, 0xc6,
-	0x29, 0xd4, 0x29, 0x3b, 0x4e, 0xb6, 0xc2, 0x20, 0xe1, 0xd3, 0x84, 0x2c, 0x83, 0xe9, 0xb9, 0xe8,
-	0x00, 0x8b, 0x9a, 0x9e, 0x2b, 0x55, 0x3e, 0x89, 0xc3, 0x49, 0x84, 0xf6, 0x6f, 0x52, 0x05, 0xa0,
-	0xa3, 0x5c, 0x37, 0x46, 0x3b, 0x48, 0x47, 0xb9, 0x6e, 0x4c, 0x5e, 0x82, 0xba, 0x08, 0x58, 0x24,
-	0x4e, 0xc3, 0x44, 0xaa, 0x5c, 0x46, 0x95, 0x21, 0x45, 0x0d, 0x84, 0xf3, 0x2b, 0x03, 0xac, 0x5d,
-	0x3e, 0x3e, 0xe2, 0xf1, 0x25, 0x29, 0xcf, 0x83, 0x8d, 0x1b, 0x0f, 0x3d, 0x57, 0x0b, 0xaa, 0x22,
-	0xdc, 0x73, 0x17, 0x8a, 0xba, 0x01, 0x96, 0xcf, 0x99, 0x74, 0xad, 0x0a, 0x7b, 0x0d, 0x49, 0x8b,
-	0xb3, 0xf1, 0xd0, 0x95, 0x3a, 0x57, 0xd4, 0x02, 0x1b, 0x6f, 0x73, 0xe6, 0xca, 0xb3, 0xf9, 0x4c,
-	0x24, 0xc3, 0x49, 0xe4, 0xb2, 0x84, 0x63, 0xaa, 0x2c, 0xcb, 0xf8, 0x15, 0xc9, 0x21, 0x62, 0xc8,
-	0x1b, 0xf0, 0xcc, 0xc8, 0x9f, 0x08, 0x99, 0xab, 0xbd, 0xe0, 0x38, 0x1c, 0x86, 0x81, 0x7f, 0x81,
-	0x5e, 0xb3, 0xe9, 0x35, 0xbd, 0xd0, 0x0b, 0x8e, 0xc3, 0xfd, 0xc0, 0xbf, 0x70, 0x7e, 0x68, 0x42,
-	0xe5, 0x21, 0x9a, 0xe1, 0x1e, 0x54, 0xc7, 0xa8, 0x50, 0x9a, 0x58, 0x3a, 0xa9, 0x3b, 0x70, 0x7d,
-	0x5d, 0x69, 0x2b, 0xba, 0x41, 0x12, 0x5f, 0xd0, 0x94, 0x54, 0x72, 0x25, 0xec, 0xc8, 0xe7, 0x89,
-	0xd0, 0xf1, 0x36, 0xc7, 0x35, 0x50, 0x8b, 0x9a, 0x4b, 0x93, 0x76, 0x3e, 0x85, 0x46, 0x71, 0x3b,
-	0x59, 0x26, 0xcf, 0xf8, 0x05, 0xda, 0xb0, 0x4c, 0xe5, 0x27, 0x79, 0x15, 0x2a, 0x98, 0x3b, 0xd0,
-	0x82, 0xf5, 0x8d, 0xe5, 0x74, 0x57, 0xc5, 0x46, 0xd5, 0xe2, 0x47, 0xe6, 0x87, 0x86, 0xdc, 0xab,
-	0x28, 0xa4, 0xb8, 0x57, 0xed, 0xea, 0xbd, 0x14, 0x5b, 0x61, 0x2f, 0xe7, 0x6f, 0x06, 0x34, 0xbe,
-	0xe4, 0x71, 0x78, 0x10, 0x87, 0x51, 0x28, 0x98, 0x4f, 0x6e, 0x81, 0xa5, 0x34, 0x7d, 0xc2, 0x39,
-	0xf4, 0xaa, 0xa4, 0x53, 0xba, 0xa1, 0x6b, 0x2f, 0xcb, 0xd0, 0xab, 0x64, 0x05, 0x60, 0xcc, 0xa6,
-	0x3b, 0x9c, 0x09, 0xde, 0x73, 0xd3, 0xb0, 0xca, 0x31, 0xa4, 0x03, 0xf6, 0x98, 0x4d, 0x07, 0xd3,
-	0x60, 0x20, 0xd0, 0xeb, 0x65, 0x9a, 0xc1, 0xe4, 0x45, 0xa8, 0x8d, 0xd9, 0x54, 0xc6, 0x77, 0xcf,
-	0xd5, 0x5e, 0xcf, 0x11, 0xe4, 0x65, 0x28, 0x25, 0xd3, 0x00, 0x73, 0x56, 0x7d, 0xe3, 0x1a, 0x5e,
-	0x8f, 0xc1, 0x34, 0xd0, 0x37, 0x81, 0xca, 0xb5, 0xd4, 0x32, 0x76, 0x66, 0x19, 0xe7, 0x97, 0x25,
-	0xb8, 0xa6, 0x1d, 0x71, 0xea, 0x45, 0xfd, 0x44, 0x46, 0x4f, 0x1b, 0xaa, 0x98, 0x0a, 0x78, 0xac,
-	0xfd, 0x91, 0x82, 0xe4, 0xff, 0xc0, 0xc2, 0x40, 0x4e, 0x5d, 0xfd, 0xca, 0xac, 0x31, 0xb2, 0x2d,
-	0x94, 0xeb, 0xb5, 0xcf, 0x35, 0x0b, 0xf9, 0x10, 0x2a, 0x5f, 0xf3, 0x38, 0x54, 0x69, 0xae, 0xbe,
-	0xe1, 0x3c, 0x89, 0x57, 0x9a, 0x5f, 0xb3, 0x2a, 0x86, 0xff, 0xa2, 0xcd, 0xd6, 0x64, 0x52, 0x1b,
-	0x87, 0xe7, 0xdc, 0x6d, 0x57, 0xf1, 0x54, 0xf3, 0xee, 0x4d, 0x97, 0x3b, 0x9f, 0x40, 0xbd, 0xa0,
-	0x54, 0x31, 0xc6, 0x9a, 0x2a, 0xc6, 0x5e, 0x99, 0x8d, 0xb1, 0xe6, 0xcc, 0x2d, 0x28, 0x86, 0xeb,
-	0x27, 0x00, 0xb9, 0x8a, 0xff, 0x4e, 0xe0, 0x3b, 0x3f, 0x30, 0xe0, 0xda, 0x56, 0x18, 0x04, 0x1c,
-	0xfb, 0x22, 0xe5, 0xbc, 0x3c, 0x5e, 0x8d, 0x2b, 0xe3, 0xf5, 0x2d, 0xa8, 0x08, 0xc9, 0xa0, 0xa5,
-	0xdc, 0x7c, 0x82, 0x37, 0xa8, 0xa2, 0x92, 0x29, 0x67, 0xcc, 0xa6, 0xc3, 0x88, 0x07, 0xae, 0x17,
-	0x9c, 0x60, 0x8c, 0x2b, 0x1f, 0x1c, 0x28, 0x8c, 0xf3, 0x13, 0x03, 0x2c, 0x15, 0xea, 0x33, 0xe9,
-	0xcf, 0x98, 0x4d, 0x7f, 0x2f, 0x42, 0x2d, 0x8a, 0xb9, 0xeb, 0x8d, 0x52, 0xc9, 0x35, 0x9a, 0x23,
-	0x64, 0x76, 0x3e, 0x0e, 0xe3, 0x11, 0xc7, 0xed, 0x6d, 0xaa, 0x00, 0xd9, 0x76, 0x62, 0xe1, 0xc1,
-	0x24, 0xa6, 0x32, 0xa4, 0x2d, 0x11, 0x32, 0x7b, 0x49, 0x16, 0x11, 0xb1, 0x91, 0x6a, 0x00, 0x4b,
-	0x54, 0x01, 0x32, 0xa3, 0x2a, 0xbf, 0x61, 0xa8, 0xdb, 0x54, 0x43, 0xce, 0x2f, 0x4c, 0x68, 0x6c,
-	0x7b, 0x31, 0x1f, 0x25, 0xdc, 0xed, 0xba, 0x27, 0x48, 0xc8, 0x83, 0xc4, 0x4b, 0x2e, 0x74, 0xf6,
-	0xd6, 0x50, 0x56, 0xba, 0xcd, 0xd9, 0xa6, 0x58, 0xf9, 0xa5, 0x84, 0xbd, 0xbc, 0x02, 0xc8, 0xfb,
-	0x00, 0xaa, 0x13, 0xc2, 0x7e, 0xbe, 0x7c, 0x75, 0x3f, 0x5f, 0x43, 0x52, 0xf9, 0x29, 0x8d, 0xa4,
-	0xf8, 0x3c, 0x95, 0xdd, 0x2d, 0x6c, 0xf6, 0x27, 0x32, 0x9c, 0xb1, 0x1f, 0x38, 0xe2, 0x3e, 0x86,
-	0x2b, 0xf6, 0x03, 0x47, 0xdc, 0xcf, 0xda, 0xb7, 0xaa, 0x3a, 0x92, 0xfc, 0x26, 0xb7, 0xc1, 0x0c,
-	0x23, 0xd4, 0xb1, 0x20, 0xb4, 0xa8, 0xe0, 0xfa, 0x7e, 0x44, 0xcd, 0x30, 0x22, 0x0e, 0x58, 0xaa,
-	0x61, 0x6d, 0xd7, 0x30, 0xcc, 0x01, 0xd3, 0x03, 0x76, 0x4c, 0x54, 0xaf, 0x38, 0x37, 0xc0, 0xdc,
-	0x8f, 0x48, 0x15, 0x4a, 0xfd, 0xee, 0xa0, 0xb5, 0x24, 0x3f, 0xb6, 0xbb, 0x3b, 0x2d, 0xc3, 0xf9,
-	0xab, 0x01, 0xb5, 0xdd, 0x49, 0xc2, 0x64, 0x8c, 0x89, 0xab, 0x9c, 0xfb, 0x3c, 0xd8, 0x22, 0x61,
-	0x31, 0xd6, 0x4b, 0x53, 0x25, 0x0e, 0x84, 0x07, 0x82, 0xbc, 0x01, 0x15, 0xee, 0x9e, 0xf0, 0xf4,
-	0xee, 0x5f, 0x5f, 0x74, 0x56, 0xaa, 0x48, 0xc8, 0x1d, 0xb0, 0xc4, 0xe8, 0x94, 0x8f, 0x59, 0xbb,
-	0x3c, 0x4b, 0xdc, 0x47, 0xac, 0x2a, 0x71, 0x54, 0xd3, 0xe0, 0xbb, 0x23, 0x0e, 0x23, 0x6c, 0xbc,
-	0x2b, 0xfa, 0xdd, 0x11, 0x87, 0x91, 0x6c, 0xbb, 0x37, 0xe0, 0x39, 0xef, 0x24, 0x08, 0x63, 0x3e,
-	0xf4, 0x02, 0x97, 0x4f, 0x87, 0xa3, 0x30, 0x38, 0xf6, 0xbd, 0x51, 0x82, 0x76, 0xb5, 0xe9, 0xb3,
-	0x6a, 0xb1, 0x27, 0xd7, 0xb6, 0xf4, 0x92, 0x73, 0x1b, 0x6a, 0x8f, 0xf8, 0x05, 0xf6, 0xaf, 0x82,
-	0x74, 0xc0, 0x3c, 0x3b, 0xd7, 0xb5, 0x10, 0xd2, 0x53, 0x3c, 0x7a, 0x4c, 0xcd, 0xb3, 0x73, 0xe7,
-	0x14, 0xec, 0xbe, 0x6e, 0x06, 0xc8, 0x5b, 0x32, 0x61, 0x62, 0x9a, 0xd5, 0x97, 0x2e, 0x7b, 0x69,
-	0x14, 0x7a, 0x11, 0x9a, 0xd2, 0x48, 0xff, 0xe2, 0x81, 0xb4, 0x91, 0x14, 0x50, 0xec, 0xaf, 0x4a,
-	0xc5, 0xfe, 0xca, 0xf9, 0xb1, 0x09, 0x76, 0x56, 0x8e, 0xde, 0x86, 0xda, 0x38, 0xf5, 0x85, 0xbe,
-	0xba, 0x59, 0xd7, 0x9d, 0x39, 0x89, 0xe6, 0x34, 0x5a, 0x87, 0xf2, 0x22, 0x1d, 0xf2, 0x1c, 0x50,
-	0x79, 0xaa, 0x1c, 0x70, 0x1b, 0xae, 0x8d, 0x7c, 0xce, 0x82, 0x61, 0x7e, 0x85, 0x55, 0x84, 0x2e,
-	0x23, 0xfa, 0x20, 0xbb, 0xc7, 0x3a, 0xa7, 0x55, 0xf3, 0x02, 0xfc, 0x3a, 0x54, 0x5c, 0xee, 0x27,
-	0x6c, 0xfe, 0x25, 0xb6, 0x1f, 0xb3, 0x91, 0xcf, 0xb7, 0xe5, 0x12, 0x55, 0x14, 0xe4, 0x0e, 0xd8,
-	0x69, 0x97, 0xa5, 0xdf, 0x5f, 0x59, 0xaf, 0x9e, 0x1a, 0x9c, 0x66, 0x14, 0xce, 0xcb, 0x50, 0x7a,
-	0xf4, 0xb8, 0x7f, 0xa5, 0xa7, 0xbe, 0x07, 0xe6, 0xa3, 0xc7, 0xc5, 0x3c, 0xdb, 0x50, 0x67, 0xd2,
-	0x2f, 0x73, 0x33, 0x7f, 0x99, 0x77, 0xc0, 0x9e, 0x08, 0x1e, 0xef, 0xf2, 0x84, 0xe9, 0x4b, 0x9e,
-	0xc1, 0xb2, 0x28, 0xca, 0xa7, 0xa5, 0x17, 0x06, 0xba, 0x00, 0xa5, 0xa0, 0xf3, 0xf7, 0x12, 0x54,
-	0xf5, 0x45, 0x97, 0x7b, 0x4e, 0xb2, 0x56, 0x50, 0x7e, 0xe6, 0x59, 0xc3, 0x2c, 0x66, 0x8d, 0xe2,
-	0x0c, 0xa0, 0xf4, 0x74, 0x33, 0x00, 0xf2, 0x1d, 0x68, 0x44, 0x6a, 0xad, 0x98, 0x6b, 0x5e, 0x98,
-	0xe7, 0xd3, 0xbf, 0xc8, 0x5b, 0x8f, 0x72, 0x40, 0xde, 0x14, 0x7c, 0x07, 0x25, 0xec, 0x04, 0x1d,
-	0xde, 0xa0, 0x55, 0x09, 0x0f, 0xd8, 0xc9, 0x13, 0x32, 0xce, 0x53, 0x24, 0x0d, 0xd9, 0xfa, 0x86,
-	0x51, 0xbb, 0x81, 0x89, 0x40, 0x26, 0x9a, 0x62, 0x0e, 0x68, 0xce, 0xe6, 0x80, 0x17, 0xa0, 0x36,
-	0x0a, 0xc7, 0x63, 0x0f, 0xd7, 0x96, 0x55, 0x99, 0x56, 0x88, 0x81, 0x70, 0xbe, 0x86, 0xaa, 0x56,
-	0x98, 0xd4, 0xa1, 0xba, 0xdd, 0x7d, 0xb0, 0x79, 0xb8, 0x23, 0xb3, 0x10, 0x80, 0x75, 0xbf, 0xb7,
-	0xb7, 0x49, 0xbf, 0x68, 0x19, 0x32, 0x23, 0xf5, 0xf6, 0x06, 0x2d, 0x93, 0xd4, 0xa0, 0xf2, 0x60,
-	0x67, 0x7f, 0x73, 0xd0, 0x2a, 0x11, 0x1b, 0xca, 0xf7, 0xf7, 0xf7, 0x77, 0x5a, 0x65, 0xd2, 0x00,
-	0x7b, 0x7b, 0x73, 0xd0, 0x1d, 0xf4, 0x76, 0xbb, 0xad, 0x8a, 0xa4, 0x7d, 0xd8, 0xdd, 0x6f, 0x59,
-	0xf2, 0xe3, 0xb0, 0xb7, 0xdd, 0xaa, 0xca, 0xf5, 0x83, 0xcd, 0x7e, 0xff, 0xf3, 0x7d, 0xba, 0xdd,
-	0xb2, 0xe5, 0xbe, 0xfd, 0x01, 0xed, 0xed, 0x3d, 0x6c, 0xd5, 0x9c, 0xbb, 0x50, 0x2f, 0x18, 0x4d,
-	0x72, 0xd0, 0xee, 0x83, 0xd6, 0x92, 0x14, 0xf3, 0x78, 0x73, 0xe7, 0xb0, 0xdb, 0x32, 0xc8, 0x32,
-	0x00, 0x7e, 0x0e, 0x77, 0x36, 0xf7, 0x1e, 0xb6, 0x4c, 0xe7, 0xfb, 0x46, 0xc6, 0x83, 0x6f, 0xeb,
-	0x37, 0xc1, 0xd6, 0xa6, 0x4e, 0x7b, 0xe7, 0x6b, 0x73, 0x7e, 0xa1, 0x19, 0x81, 0x0c, 0xb3, 0xd1,
-	0x29, 0x1f, 0x9d, 0x89, 0xc9, 0x58, 0x47, 0x45, 0x06, 0xab, 0x27, 0xb2, 0xb4, 0x49, 0x9a, 0x04,
-	0x14, 0x94, 0xcd, 0x99, 0xca, 0x48, 0xaf, 0xe6, 0x4c, 0xf7, 0x00, 0xf2, 0x49, 0xc6, 0x82, 0xae,
-	0xf7, 0x3a, 0x54, 0x98, 0xef, 0x31, 0xa1, 0xab, 0x98, 0x02, 0x1c, 0x0a, 0xf5, 0xc2, 0xfc, 0x43,
-	0x3a, 0x8c, 0xf9, 0xfe, 0xf0, 0x8c, 0x5f, 0x08, 0xe4, 0xb5, 0x69, 0x95, 0xf9, 0xfe, 0x23, 0x7e,
-	0x21, 0xc8, 0x1a, 0x54, 0xd4, 0xf8, 0xc4, 0x5c, 0xf0, 0xd0, 0x46, 0x76, 0xaa, 0x08, 0x9c, 0x3b,
-	0x60, 0xa9, 0xd7, 0x77, 0x21, 0x66, 0x8c, 0x27, 0x16, 0x9a, 0x8f, 0xf5, 0xb9, 0xf1, 0xad, 0x4e,
-	0xde, 0xd6, 0xa3, 0x1a, 0xa1, 0x06, 0x44, 0xc6, 0x6c, 0x1b, 0xa6, 0x08, 0xf5, 0x94, 0x06, 0x19,
-	0x9c, 0x6d, 0xb0, 0xaf, 0x1c, 0x84, 0x69, 0x43, 0x98, 0xb9, 0x21, 0x16, 0x8c, 0xc6, 0x9c, 0x18,
-	0x20, 0x1f, 0xe7, 0xe8, 0x30, 0x56, 0xbb, 0xc8, 0x30, 0x5e, 0x97, 0x2e, 0xf2, 0x7c, 0x37, 0xe6,
-	0xc1, 0x25, 0xed, 0xf3, 0x21, 0x50, 0x46, 0x43, 0x5e, 0x85, 0x32, 0x4e, 0xad, 0x4a, 0xb3, 0x09,
-	0x2b, 0x1b, 0x59, 0xe1, 0xaa, 0x73, 0x04, 0x4d, 0x55, 0xc3, 0x28, 0xff, 0x6a, 0xc2, 0xc5, 0x95,
-	0x9d, 0xd2, 0x0a, 0x40, 0x96, 0x66, 0xd3, 0x39, 0x5c, 0x01, 0x23, 0x03, 0xe5, 0xd8, 0xe3, 0xbe,
-	0x9b, 0x6a, 0xa5, 0x21, 0xe7, 0x03, 0x68, 0xa4, 0x32, 0xf0, 0x99, 0x7d, 0x3b, 0xab, 0xa6, 0x69,
-	0x5c, 0x4a, 0x87, 0x28, 0x92, 0xbd, 0xd0, 0xcd, 0x0a, 0xa9, 0xf3, 0x67, 0x33, 0xe5, 0xd4, 0x8f,
-	0xc8, 0x99, 0x5e, 0xcd, 0x98, 0xef, 0xd5, 0x66, 0xfb, 0x1e, 0xf3, 0xa9, 0xfb, 0x9e, 0xff, 0x87,
-	0x9a, 0x8b, 0x45, 0xdf, 0x3b, 0x4f, 0x53, 0xdf, 0xca, 0xa2, 0x02, 0xaf, 0x5b, 0x03, 0xef, 0x9c,
-	0xd3, 0x9c, 0x41, 0x9e, 0x29, 0x09, 0xcf, 0x78, 0xe0, 0x7d, 0x8d, 0xaf, 0x65, 0xa9, 0x78, 0x8e,
-	0xc8, 0x07, 0x1a, 0xaa, 0x11, 0xd0, 0x03, 0x8d, 0x74, 0x98, 0x63, 0x15, 0x86, 0x39, 0x37, 0xc0,
-	0x9a, 0x44, 0x82, 0xc7, 0x49, 0xda, 0x20, 0x2a, 0x28, 0x6b, 0xb2, 0x6a, 0x9a, 0x96, 0x05, 0x27,
-	0xce, 0xff, 0x42, 0x2d, 0x3b, 0x8b, 0xcc, 0x37, 0x7b, 0xfb, 0x7b, 0x5d, 0x95, 0x1d, 0x7a, 0x7b,
-	0xdb, 0xdd, 0xef, 0xb6, 0x0c, 0x99, 0xb1, 0x68, 0xf7, 0x71, 0x97, 0xf6, 0xbb, 0x2d, 0x53, 0x66,
-	0x96, 0xed, 0xee, 0x4e, 0x77, 0xd0, 0x6d, 0x95, 0x3e, 0x2d, 0xdb, 0xd5, 0x96, 0x4d, 0x6d, 0x3e,
-	0x8d, 0x7c, 0x6f, 0xe4, 0x25, 0xce, 0x17, 0x60, 0xef, 0xb2, 0xe8, 0x52, 0xe3, 0x9f, 0x17, 0xa4,
-	0x89, 0x9e, 0x18, 0xe8, 0xe2, 0xf1, 0x3a, 0x54, 0x75, 0xd6, 0xd0, 0x91, 0x75, 0x29, 0xab, 0xa4,
-	0xeb, 0xce, 0xcf, 0x0d, 0xb8, 0xbe, 0x1b, 0x9e, 0xf3, 0xac, 0x0a, 0x1f, 0xb0, 0x0b, 0x3f, 0x64,
-	0xee, 0xb7, 0xb8, 0xf1, 0x16, 0x5c, 0x13, 0xe1, 0x24, 0x1e, 0xf1, 0xe1, 0xdc, 0xc4, 0xa2, 0xa9,
-	0xd0, 0x0f, 0x75, 0x38, 0x3a, 0xd0, 0x74, 0xb9, 0x48, 0x72, 0xaa, 0x12, 0x52, 0xd5, 0x25, 0x32,
-	0xa5, 0xc9, 0xda, 0x89, 0xf2, 0xd3, 0xb4, 0x13, 0xce, 0x6f, 0x0d, 0x68, 0x76, 0xa7, 0x51, 0x18,
-	0x27, 0xe9, 0x51, 0x9f, 0x93, 0x6d, 0xfb, 0x57, 0xe9, 0x65, 0x28, 0xd3, 0x4a, 0xcc, 0xbf, 0xea,
-	0x5d, 0x39, 0x4e, 0xb9, 0x07, 0x96, 0xdc, 0x6c, 0x22, 0x74, 0x28, 0xbd, 0x98, 0xca, 0x9c, 0xd9,
-	0x78, 0xbd, 0x8f, 0x34, 0x54, 0xd3, 0x16, 0x5b, 0xad, 0xf2, 0x4c, 0xab, 0xf5, 0x11, 0x58, 0x8a,
-	0xb4, 0xe0, 0xe7, 0x3a, 0x54, 0xfb, 0x87, 0x5b, 0x5b, 0xdd, 0x7e, 0xbf, 0x65, 0x90, 0x26, 0xd4,
-	0xb6, 0x0f, 0x0f, 0x76, 0x7a, 0x5b, 0x9b, 0x03, 0xed, 0xeb, 0x07, 0x9b, 0xbd, 0x9d, 0xee, 0x76,
-	0xab, 0xe4, 0x6c, 0x41, 0x6d, 0x30, 0x0d, 0x34, 0x7b, 0xb1, 0x0c, 0x1a, 0x57, 0x94, 0x41, 0x73,
-	0xae, 0x0c, 0x7e, 0x0e, 0xf5, 0x42, 0x4b, 0x44, 0x5e, 0x83, 0x72, 0x32, 0x0d, 0x2e, 0xcf, 0x79,
-	0x53, 0x39, 0x14, 0x97, 0xc9, 0xcb, 0xd0, 0x90, 0x8f, 0x33, 0x26, 0x84, 0x77, 0x12, 0x70, 0x57,
-	0xef, 0x2a, 0x1f, 0x6c, 0x9b, 0x1a, 0xe5, 0xbc, 0x04, 0x4d, 0xf9, 0x1e, 0xf6, 0xc6, 0x5c, 0x24,
-	0x6c, 0x1c, 0x61, 0xe1, 0xd6, 0x49, 0xba, 0x4c, 0xcd, 0x44, 0x38, 0xb7, 0xa0, 0x71, 0xc0, 0x79,
-	0x4c, 0xb9, 0x88, 0xc2, 0x40, 0xe0, 0xcb, 0x48, 0x5b, 0x56, 0x55, 0x05, 0x0d, 0x39, 0x47, 0x50,
-	0x93, 0x4d, 0xed, 0x7d, 0x96, 0x8c, 0x4e, 0xff, 0xd9, 0xc6, 0xf7, 0x16, 0x54, 0x23, 0xe5, 0x11,
-	0xdd, 0xba, 0xaa, 0x21, 0x9e, 0xf6, 0x12, 0x4d, 0x17, 0x9d, 0x9b, 0x50, 0xda, 0x9b, 0x8c, 0x8b,
-	0x7f, 0x9d, 0x94, 0xb1, 0x41, 0x73, 0x1e, 0x40, 0x23, 0xed, 0x01, 0xb1, 0x29, 0x93, 0xb6, 0xf4,
-	0x3d, 0x1e, 0x14, 0xec, 0x6c, 0x2b, 0xc4, 0x40, 0x5c, 0x11, 0x36, 0x1b, 0xbf, 0x31, 0xa0, 0x2c,
-	0x4f, 0x28, 0xf3, 0x76, 0x77, 0x74, 0x1a, 0x92, 0x99, 0x83, 0x74, 0x66, 0x20, 0x67, 0x89, 0xbc,
-	0xa3, 0x86, 0x8a, 0xe9, 0x24, 0xf6, 0x99, 0xa2, 0x92, 0x68, 0x88, 0x4b, 0x1c, 0x1b, 0x50, 0xff,
-	0x34, 0xf4, 0x82, 0x2d, 0x35, 0x6b, 0x23, 0x8b, 0xcc, 0x72, 0x89, 0xe7, 0x3d, 0xb0, 0x7a, 0x42,
-	0xfa, 0x60, 0x31, 0x79, 0xf6, 0x0c, 0x2a, 0xba, 0xc9, 0x59, 0xda, 0xf8, 0x69, 0x09, 0xca, 0x5f,
-	0xf2, 0x38, 0x24, 0xf7, 0xa0, 0xaa, 0x87, 0x01, 0x64, 0xee, 0xd1, 0xdf, 0xc9, 0xae, 0xe2, 0xdc,
-	0xb4, 0xc0, 0x59, 0x22, 0xef, 0x83, 0xa5, 0xf3, 0xfd, 0xec, 0xc4, 0xa2, 0xf3, 0xa4, 0xeb, 0xeb,
-	0x2c, 0xad, 0x19, 0xef, 0x18, 0xe4, 0x6d, 0xb0, 0x54, 0xa4, 0xce, 0xd9, 0x6e, 0x51, 0x6b, 0xef,
-	0x2c, 0x21, 0x43, 0xbd, 0x7f, 0x1a, 0x4e, 0x7c, 0xb7, 0xcf, 0xe3, 0x73, 0x4e, 0xe6, 0xe6, 0x63,
-	0x9d, 0x39, 0xd8, 0x59, 0x22, 0x6f, 0x01, 0xa8, 0xf0, 0x3d, 0xf4, 0x5c, 0x41, 0xea, 0xe9, 0xfa,
-	0xde, 0x64, 0xdc, 0x69, 0xa1, 0xc8, 0x34, 0xb8, 0x7b, 0xae, 0x50, 0xe4, 0x85, 0xf0, 0xfe, 0x56,
-	0xf2, 0x77, 0xa1, 0xb9, 0x85, 0xb7, 0x6e, 0x3f, 0xde, 0x3c, 0x0a, 0xe3, 0x84, 0xcc, 0x4f, 0xcc,
-	0x3a, 0xf3, 0x08, 0x67, 0x89, 0x7c, 0x04, 0xf6, 0x20, 0xbe, 0x50, 0xf4, 0xcf, 0x15, 0x6e, 0x63,
-	0x2e, 0xb8, 0xb3, 0x18, 0xed, 0x2c, 0x6d, 0xfc, 0xb1, 0x04, 0xd6, 0xe7, 0x61, 0x7c, 0xc6, 0x63,
-	0xb2, 0x0e, 0x16, 0xbe, 0xd5, 0x0a, 0xa1, 0x94, 0xbd, 0xdd, 0x16, 0x89, 0xbd, 0x03, 0x35, 0x34,
-	0xda, 0x80, 0x89, 0xb3, 0xdc, 0x4d, 0xf8, 0xf7, 0x5d, 0x6e, 0x37, 0x55, 0xef, 0x51, 0xb3, 0xe5,
-	0x7e, 0x12, 0x73, 0x36, 0xce, 0xde, 0xa7, 0x97, 0x1e, 0x50, 0x9d, 0x7a, 0xfe, 0x46, 0xea, 0xa3,
-	0x77, 0xee, 0x42, 0xb9, 0x2f, 0xb5, 0xca, 0xff, 0x70, 0xcb, 0xff, 0x7a, 0xe8, 0x90, 0x22, 0x32,
-	0x93, 0xf3, 0x01, 0x58, 0xaa, 0x60, 0xe7, 0xa6, 0x98, 0xe9, 0x6e, 0x3a, 0xd7, 0xe7, 0xd1, 0x9a,
-	0xf1, 0x16, 0x54, 0x0f, 0x26, 0xf1, 0x09, 0x1f, 0x88, 0xb9, 0xd8, 0x29, 0x3a, 0xcd, 0x59, 0x22,
-	0x1f, 0x82, 0xa5, 0xd2, 0x78, 0x2e, 0x60, 0x26, 0xad, 0x77, 0x16, 0xa3, 0x9d, 0x25, 0x72, 0x17,
-	0x5a, 0x94, 0x8f, 0xb8, 0x57, 0x28, 0x87, 0xa4, 0xa8, 0xf2, 0xfc, 0xdd, 0x5b, 0x33, 0xc8, 0xc7,
-	0xd0, 0x9c, 0x29, 0x9f, 0x24, 0x2b, 0x25, 0x8b, 0xaa, 0xea, 0xfc, 0x06, 0xf7, 0x5b, 0xbf, 0xfe,
-	0x66, 0xc5, 0xf8, 0xfd, 0x37, 0x2b, 0xc6, 0x9f, 0xbe, 0x59, 0x31, 0x7e, 0xf4, 0x97, 0x95, 0xa5,
-	0x23, 0x0b, 0xff, 0x28, 0x7e, 0xf7, 0x1f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xc7, 0xde, 0xd3, 0xd9,
-	0x4d, 0x1e, 0x00, 0x00,
+	0xf4, 0xea, 0xe5, 0x26, 0xb5, 0x8e, 0x7d, 0x08, 0xd1, 0xe2, 0xb4, 0xa8, 0x81, 0x86, 0x33, 0xe3,
+	0xe9, 0xa1, 0x40, 0xf9, 0x98, 0x5b, 0x90, 0x3f, 0x10, 0xe4, 0x98, 0x5c, 0x03, 0x04, 0xc8, 0x31,
+	0xf7, 0x00, 0x09, 0x10, 0x20, 0xc9, 0x39, 0x97, 0xc4, 0xb9, 0x05, 0xc8, 0x21, 0xff, 0x20, 0xe8,
+	0xea, 0x9e, 0x07, 0x29, 0xad, 0xbc, 0x79, 0x9d, 0x38, 0xd5, 0x5d, 0xd5, 0xd5, 0xf5, 0xe8, 0x7a,
+	0x11, 0x96, 0xbd, 0x20, 0xe1, 0x71, 0xc0, 0xfc, 0xf5, 0x28, 0x0e, 0x93, 0x90, 0x58, 0x0a, 0xee,
+	0xd4, 0x58, 0xe4, 0xa9, 0x25, 0xa7, 0x03, 0xe5, 0x1d, 0x4f, 0x24, 0x84, 0x40, 0x79, 0xea, 0xb9,
+	0xa2, 0x6d, 0xac, 0x96, 0xd6, 0x2c, 0x8a, 0xdf, 0xce, 0x67, 0x50, 0x1b, 0x30, 0x71, 0xfa, 0x98,
+	0xf9, 0x53, 0x4e, 0x5a, 0x50, 0x3a, 0x63, 0x7e, 0xdb, 0x58, 0x35, 0xd6, 0x1a, 0x54, 0x7e, 0x92,
+	0x0d, 0xb0, 0xcf, 0x98, 0x3f, 0x4c, 0xce, 0x23, 0xde, 0x36, 0x57, 0x8d, 0xb5, 0xe5, 0x8d, 0x9b,
+	0xeb, 0x8a, 0xc1, 0xfa, 0x41, 0x28, 0x12, 0x2f, 0x18, 0xaf, 0x3f, 0x66, 0xfe, 0xe0, 0x3c, 0xe2,
+	0xb4, 0x7a, 0xa6, 0x3e, 0x9c, 0x7d, 0xa8, 0xf7, 0xe3, 0xd1, 0x83, 0x69, 0x30, 0x4a, 0xbc, 0x30,
+	0x90, 0x5c, 0x03, 0x36, 0xe1, 0x78, 0x6a, 0x8d, 0xe2, 0xb7, 0x5c, 0x63, 0xf1, 0x58, 0xb4, 0x4b,
+	0xab, 0x25, 0xb9, 0x26, 0xbf, 0x49, 0x1b, 0xaa, 0x9e, 0xd8, 0x0a, 0xa7, 0x41, 0xd2, 0x2e, 0xaf,
+	0x1a, 0x6b, 0x36, 0x4d, 0x41, 0xe7, 0x67, 0x25, 0xa8, 0x7c, 0x36, 0xe5, 0xf1, 0x39, 0xd2, 0x25,
+	0x49, 0x9c, 0x9e, 0x25, 0xbf, 0xc9, 0x75, 0xa8, 0xf8, 0x2c, 0x18, 0x8b, 0xb6, 0x89, 0x87, 0x29,
+	0x80, 0xbc, 0x00, 0x35, 0x76, 0x9c, 0xf0, 0x78, 0x38, 0xf5, 0xdc, 0x76, 0x69, 0xd5, 0x58, 0xb3,
+	0xa8, 0x8d, 0x0b, 0x87, 0x9e, 0x4b, 0x9e, 0x07, 0xdb, 0x0d, 0x87, 0xa3, 0x22, 0x2f, 0x37, 0x44,
+	0x5e, 0xe4, 0x36, 0xd8, 0x53, 0xcf, 0x1d, 0xfa, 0x9e, 0x48, 0xda, 0x95, 0x55, 0x63, 0xad, 0xbe,
+	0xd1, 0x48, 0x05, 0x96, 0x3a, 0xa4, 0xd5, 0xa9, 0xe7, 0xa2, 0x32, 0xd7, 0xc1, 0x16, 0xf1, 0x68,
+	0x78, 0x3c, 0x0d, 0x46, 0x6d, 0x0b, 0x11, 0x9f, 0x4d, 0x11, 0x0b, 0xd2, 0xd3, 0xaa, 0x50, 0x80,
+	0x14, 0x2f, 0xe6, 0x67, 0x3c, 0x16, 0xbc, 0x5d, 0x55, 0x2c, 0x35, 0x48, 0xee, 0x41, 0xfd, 0x98,
+	0x8d, 0x78, 0x32, 0x8c, 0x58, 0xcc, 0x26, 0x6d, 0x7b, 0xfe, 0xb0, 0x07, 0x72, 0xeb, 0x40, 0xee,
+	0x08, 0x0a, 0xc7, 0x19, 0x40, 0x3e, 0x80, 0x26, 0x42, 0x62, 0x78, 0xec, 0xf9, 0x09, 0x8f, 0xdb,
+	0x35, 0xa4, 0x23, 0x19, 0x1d, 0xae, 0x0e, 0x62, 0xce, 0x69, 0x43, 0x21, 0xaa, 0x15, 0xf2, 0x3f,
+	0x00, 0x7c, 0x16, 0xb1, 0xc0, 0x1d, 0x32, 0xdf, 0x6f, 0x03, 0xde, 0xa5, 0xa6, 0x56, 0x36, 0x7d,
+	0x9f, 0xdc, 0x94, 0xf7, 0x64, 0xee, 0x30, 0x11, 0xed, 0xe6, 0xaa, 0xb1, 0x56, 0xa6, 0x96, 0x04,
+	0x07, 0x42, 0x6a, 0xc6, 0xf7, 0x82, 0xa1, 0x84, 0xda, 0xcb, 0x5a, 0x33, 0xd2, 0xc7, 0x76, 0xbc,
+	0x80, 0x72, 0xe6, 0xd2, 0xaa, 0xaf, 0x3e, 0x9c, 0xf7, 0xa1, 0x86, 0xee, 0x84, 0x6a, 0x7a, 0x1d,
+	0xac, 0x33, 0x09, 0x28, 0xaf, 0xab, 0x6f, 0x3c, 0x93, 0xde, 0x2f, 0xf3, 0x3a, 0xaa, 0x11, 0x9c,
+	0x15, 0xb0, 0x77, 0x58, 0x30, 0x4e, 0x5d, 0x55, 0xda, 0x11, 0x89, 0x6a, 0x14, 0xbf, 0x9d, 0x3f,
+	0x98, 0x60, 0x51, 0x2e, 0xa6, 0x7e, 0x42, 0xde, 0x04, 0x90, 0x56, 0x9a, 0xb0, 0x24, 0xf6, 0x66,
+	0xfa, 0xe4, 0x79, 0x3b, 0xd5, 0xa6, 0x9e, 0xbb, 0x8b, 0xdb, 0xe4, 0x1e, 0x34, 0x90, 0x43, 0x8a,
+	0x6e, 0xce, 0x5f, 0x24, 0xbb, 0x2b, 0xad, 0x23, 0x9a, 0xa6, 0xba, 0x01, 0x16, 0x3a, 0x88, 0x72,
+	0xd2, 0x26, 0xd5, 0x10, 0x79, 0x4d, 0xbf, 0x38, 0xc1, 0x47, 0xc9, 0xd0, 0xe5, 0x22, 0xf5, 0xa0,
+	0x66, 0xb6, 0xba, 0xcd, 0x45, 0x42, 0xde, 0x03, 0xa5, 0xf5, 0x94, 0x69, 0x05, 0x99, 0x92, 0x39,
+	0xab, 0x0a, 0xc5, 0x15, 0xf1, 0x34, 0xd7, 0xbb, 0x50, 0x97, 0xb2, 0xa6, 0x54, 0x16, 0x52, 0xb5,
+	0x32, 0xc9, 0xb4, 0x7a, 0x28, 0x48, 0x24, 0x4d, 0x22, 0x55, 0x25, 0xbd, 0x55, 0x79, 0x15, 0x7e,
+	0x3f, 0xbd, 0xad, 0xba, 0x50, 0xd9, 0x8f, 0x5d, 0x1e, 0x5f, 0xfa, 0xb2, 0x08, 0x94, 0x5d, 0x2e,
+	0x46, 0xf8, 0xf0, 0x6d, 0x8a, 0xdf, 0xf9, 0x6b, 0x2b, 0x15, 0x5e, 0x9b, 0xf3, 0x3b, 0x03, 0xea,
+	0xfd, 0x30, 0x4e, 0x76, 0xb9, 0x10, 0x6c, 0xcc, 0xc9, 0x2b, 0x50, 0x09, 0xe5, 0xb1, 0xda, 0x34,
+	0xcd, 0x54, 0x00, 0xe4, 0x45, 0xd5, 0xde, 0x82, 0x11, 0xcd, 0xab, 0x8d, 0x78, 0x1d, 0x2a, 0xea,
+	0xbd, 0xca, 0xb7, 0x5c, 0xa1, 0x0a, 0x90, 0x46, 0x0a, 0x8f, 0x8f, 0x05, 0x57, 0x46, 0xa8, 0x50,
+	0x0d, 0xfd, 0x07, 0x9c, 0xf8, 0x08, 0x40, 0x0a, 0xf4, 0xaf, 0xf8, 0xdb, 0x53, 0xf3, 0x38, 0x81,
+	0x3a, 0x65, 0xc7, 0xc9, 0x56, 0x18, 0x24, 0x7c, 0x96, 0x90, 0x65, 0x30, 0x3d, 0x17, 0x0d, 0x60,
+	0x51, 0xd3, 0x73, 0xa5, 0xc8, 0xe3, 0x38, 0x9c, 0x46, 0xa8, 0xff, 0x26, 0x55, 0x00, 0x1a, 0xca,
+	0x75, 0x63, 0xd4, 0x83, 0x34, 0x94, 0xeb, 0xc6, 0xe4, 0x25, 0xa8, 0x8b, 0x80, 0x45, 0xe2, 0x24,
+	0x4c, 0xa4, 0xc8, 0x65, 0x14, 0x19, 0xd2, 0xa5, 0x81, 0x70, 0x7e, 0x65, 0x80, 0xb5, 0xcb, 0x27,
+	0x47, 0x3c, 0xbe, 0xc0, 0xe5, 0x79, 0xb0, 0xf1, 0xe0, 0xa1, 0xe7, 0x6a, 0x46, 0x55, 0x84, 0x7b,
+	0xee, 0xa5, 0xac, 0x6e, 0x80, 0xe5, 0x73, 0x26, 0x4d, 0xab, 0xdc, 0x5e, 0x43, 0x52, 0xe3, 0x6c,
+	0x32, 0x74, 0xa5, 0xcc, 0x15, 0xb5, 0xc1, 0x26, 0xdb, 0x9c, 0xb9, 0xf2, 0x6e, 0x3e, 0x13, 0xc9,
+	0x70, 0x1a, 0xb9, 0x2c, 0xe1, 0x18, 0x2a, 0xcb, 0xd2, 0x7f, 0x45, 0x72, 0x88, 0x2b, 0xe4, 0x0d,
+	0x78, 0x66, 0xe4, 0x4f, 0x85, 0x8c, 0xd5, 0x5e, 0x70, 0x1c, 0x0e, 0xc3, 0xc0, 0x3f, 0x47, 0xab,
+	0xd9, 0xf4, 0x9a, 0xde, 0xe8, 0x05, 0xc7, 0xe1, 0x7e, 0xe0, 0x9f, 0x3b, 0x3f, 0x34, 0xa1, 0xf2,
+	0x10, 0xd5, 0x70, 0x0f, 0xaa, 0x13, 0x14, 0x28, 0x0d, 0x2c, 0x9d, 0xd4, 0x1c, 0xb8, 0xbf, 0xae,
+	0xa4, 0x15, 0xdd, 0x20, 0x89, 0xcf, 0x69, 0x8a, 0x2a, 0xa9, 0x12, 0x76, 0xe4, 0xf3, 0x44, 0x68,
+	0x7f, 0x5b, 0xa0, 0x1a, 0xa8, 0x4d, 0x4d, 0xa5, 0x51, 0x3b, 0x9f, 0x42, 0xa3, 0x78, 0x9c, 0x4c,
+	0x93, 0xa7, 0xfc, 0x1c, 0x75, 0x58, 0xa6, 0xf2, 0x93, 0xbc, 0x0a, 0x15, 0x8c, 0x1d, 0xa8, 0xc1,
+	0xfa, 0xc6, 0x72, 0x7a, 0xaa, 0x22, 0xa3, 0x6a, 0xf3, 0x23, 0xf3, 0x43, 0x43, 0x9e, 0x55, 0x64,
+	0x52, 0x3c, 0xab, 0x76, 0xf5, 0x59, 0x8a, 0xac, 0x70, 0x96, 0xf3, 0x37, 0x03, 0x1a, 0x5f, 0xf2,
+	0x38, 0x3c, 0x88, 0xc3, 0x28, 0x14, 0xcc, 0x27, 0xb7, 0xc0, 0x52, 0x92, 0x3e, 0xe1, 0x1e, 0x7a,
+	0x57, 0xe2, 0x29, 0xd9, 0xd0, 0xb4, 0x17, 0x79, 0xe8, 0x5d, 0xb2, 0x02, 0x30, 0x61, 0xb3, 0x1d,
+	0xce, 0x04, 0xef, 0xb9, 0xa9, 0x5b, 0xe5, 0x2b, 0xa4, 0x03, 0xf6, 0x84, 0xcd, 0x06, 0xb3, 0x60,
+	0x20, 0xd0, 0xea, 0x65, 0x9a, 0xc1, 0xe4, 0x45, 0xa8, 0x4d, 0xd8, 0x4c, 0xfa, 0x77, 0xcf, 0xd5,
+	0x56, 0xcf, 0x17, 0xc8, 0xcb, 0x50, 0x4a, 0x66, 0x01, 0xc6, 0xac, 0xfa, 0xc6, 0x35, 0x7c, 0x1e,
+	0x83, 0x59, 0xa0, 0x5f, 0x02, 0x95, 0x7b, 0xa9, 0x66, 0xec, 0x4c, 0x33, 0xce, 0x2f, 0x4b, 0x70,
+	0x4d, 0x1b, 0xe2, 0xc4, 0x8b, 0xfa, 0x89, 0xf4, 0x9e, 0x36, 0x54, 0x31, 0x14, 0xf0, 0x58, 0xdb,
+	0x23, 0x05, 0xc9, 0xff, 0x81, 0x85, 0x8e, 0x9c, 0x9a, 0xfa, 0x95, 0x79, 0x65, 0x64, 0x47, 0x28,
+	0xd3, 0x6b, 0x9b, 0x6b, 0x12, 0xf2, 0x21, 0x54, 0xbe, 0xe6, 0x71, 0xa8, 0xc2, 0x5c, 0x7d, 0xc3,
+	0x79, 0x12, 0xad, 0x54, 0xbf, 0x26, 0x55, 0x04, 0xff, 0x45, 0x9d, 0xad, 0xc9, 0xa0, 0x36, 0x09,
+	0xcf, 0xb8, 0xdb, 0xae, 0xe2, 0xad, 0x16, 0xcd, 0x9b, 0x6e, 0x77, 0x3e, 0x81, 0x7a, 0x41, 0xa8,
+	0xa2, 0x8f, 0x35, 0x95, 0x8f, 0xbd, 0x32, 0xef, 0x63, 0xcd, 0xb9, 0x57, 0x50, 0x74, 0xd7, 0x4f,
+	0x00, 0x72, 0x11, 0xff, 0x1d, 0xc7, 0x77, 0x7e, 0x60, 0xc0, 0xb5, 0xad, 0x30, 0x08, 0x38, 0xd6,
+	0x45, 0xca, 0x78, 0xb9, 0xbf, 0x1a, 0x57, 0xfa, 0xeb, 0x5b, 0x50, 0x11, 0x92, 0x40, 0x73, 0xb9,
+	0xf9, 0x04, 0x6b, 0x50, 0x85, 0x25, 0x43, 0xce, 0x84, 0xcd, 0x86, 0x11, 0x0f, 0x5c, 0x2f, 0x18,
+	0xa3, 0x8f, 0x2b, 0x1b, 0x1c, 0xa8, 0x15, 0xe7, 0x27, 0x06, 0x58, 0xca, 0xd5, 0xe7, 0xc2, 0x9f,
+	0x31, 0x1f, 0xfe, 0x5e, 0x84, 0x5a, 0x14, 0x73, 0xd7, 0x1b, 0xa5, 0x9c, 0x6b, 0x34, 0x5f, 0x90,
+	0xd1, 0xf9, 0x38, 0x8c, 0x47, 0x1c, 0x8f, 0xb7, 0xa9, 0x02, 0x64, 0xd9, 0x89, 0x89, 0x07, 0x83,
+	0x98, 0x8a, 0x90, 0xb6, 0x5c, 0x90, 0xd1, 0x4b, 0x92, 0x88, 0x88, 0x8d, 0x54, 0x01, 0x58, 0xa2,
+	0x0a, 0x90, 0x11, 0x55, 0xd9, 0x0d, 0x5d, 0xdd, 0xa6, 0x1a, 0x72, 0x7e, 0x61, 0x42, 0x63, 0xdb,
+	0x8b, 0xf9, 0x28, 0xe1, 0x6e, 0xd7, 0x1d, 0x23, 0x22, 0x0f, 0x12, 0x2f, 0x39, 0xd7, 0xd1, 0x5b,
+	0x43, 0x59, 0xea, 0x36, 0xe7, 0x8b, 0x62, 0x65, 0x97, 0x12, 0xd6, 0xf2, 0x0a, 0x20, 0xef, 0x03,
+	0xa8, 0x4a, 0x08, 0xeb, 0xf9, 0xf2, 0xd5, 0xf5, 0x7c, 0x0d, 0x51, 0xe5, 0xa7, 0x54, 0x92, 0xa2,
+	0xf3, 0x54, 0x74, 0xb7, 0xb0, 0xd8, 0x9f, 0x4a, 0x77, 0xc6, 0x7a, 0xe0, 0x88, 0xfb, 0xe8, 0xae,
+	0x58, 0x0f, 0x1c, 0x71, 0x3f, 0x2b, 0xdf, 0xaa, 0xea, 0x4a, 0xf2, 0x9b, 0xdc, 0x06, 0x33, 0x8c,
+	0x50, 0xc6, 0x02, 0xd3, 0xa2, 0x80, 0xeb, 0xfb, 0x11, 0x35, 0xc3, 0x88, 0x38, 0x60, 0xa9, 0x82,
+	0xb5, 0x5d, 0x43, 0x37, 0x07, 0x0c, 0x0f, 0x58, 0x31, 0x51, 0xbd, 0xe3, 0xdc, 0x00, 0x73, 0x3f,
+	0x22, 0x55, 0x28, 0xf5, 0xbb, 0x83, 0xd6, 0x92, 0xfc, 0xd8, 0xee, 0xee, 0xb4, 0x0c, 0xe7, 0xaf,
+	0x06, 0xd4, 0x76, 0xa7, 0x09, 0x93, 0x3e, 0x26, 0xae, 0x32, 0xee, 0xf3, 0x60, 0x8b, 0x84, 0xc5,
+	0x98, 0x2f, 0x4d, 0x15, 0x38, 0x10, 0x1e, 0x08, 0xf2, 0x06, 0x54, 0xb8, 0x3b, 0xe6, 0xe9, 0xdb,
+	0xbf, 0x7e, 0xd9, 0x5d, 0xa9, 0x42, 0x21, 0x77, 0xc0, 0x12, 0xa3, 0x13, 0x3e, 0x61, 0xed, 0xf2,
+	0x3c, 0x72, 0x1f, 0x57, 0x55, 0x8a, 0xa3, 0x1a, 0x07, 0xfb, 0x8e, 0x38, 0x8c, 0xb0, 0xf0, 0xae,
+	0xe8, 0xbe, 0x23, 0x0e, 0x23, 0x59, 0x76, 0x6f, 0xc0, 0x73, 0xde, 0x38, 0x08, 0x63, 0x3e, 0xf4,
+	0x02, 0x97, 0xcf, 0x86, 0xa3, 0x30, 0x38, 0xf6, 0xbd, 0x51, 0x82, 0x7a, 0xb5, 0xe9, 0xb3, 0x6a,
+	0xb3, 0x27, 0xf7, 0xb6, 0xf4, 0x96, 0x73, 0x1b, 0x6a, 0x8f, 0xf8, 0x39, 0xd6, 0xaf, 0x82, 0x74,
+	0xc0, 0x3c, 0x3d, 0xd3, 0xb9, 0x10, 0xd2, 0x5b, 0x3c, 0x7a, 0x4c, 0xcd, 0xd3, 0x33, 0xe7, 0x04,
+	0xec, 0xbe, 0x2e, 0x06, 0xc8, 0x5b, 0x32, 0x60, 0x62, 0x98, 0xd5, 0x8f, 0x2e, 0xeb, 0x34, 0x0a,
+	0xb5, 0x08, 0x4d, 0x71, 0xa4, 0x7d, 0xf1, 0x42, 0x5a, 0x49, 0x0a, 0x28, 0xd6, 0x57, 0xa5, 0x62,
+	0x7d, 0xe5, 0xfc, 0xd8, 0x04, 0x3b, 0x4b, 0x47, 0x6f, 0x43, 0x6d, 0x92, 0xda, 0x42, 0x3f, 0xdd,
+	0xac, 0xea, 0xce, 0x8c, 0x44, 0x73, 0x1c, 0x2d, 0x43, 0xf9, 0x32, 0x19, 0xf2, 0x18, 0x50, 0x79,
+	0xaa, 0x18, 0x70, 0x1b, 0xae, 0x8d, 0x7c, 0xce, 0x82, 0x61, 0xfe, 0x84, 0x95, 0x87, 0x2e, 0xe3,
+	0xf2, 0x41, 0xf6, 0x8e, 0x75, 0x4c, 0xab, 0xe6, 0x09, 0xf8, 0x75, 0xa8, 0xb8, 0xdc, 0x4f, 0xd8,
+	0x62, 0x27, 0xb6, 0x1f, 0xb3, 0x91, 0xcf, 0xb7, 0xe5, 0x16, 0x55, 0x18, 0xe4, 0x0e, 0xd8, 0x69,
+	0x95, 0xa5, 0xfb, 0xaf, 0xac, 0x56, 0x4f, 0x15, 0x4e, 0x33, 0x0c, 0xe7, 0x65, 0x28, 0x3d, 0x7a,
+	0xdc, 0xbf, 0xd2, 0x52, 0xdf, 0x03, 0xf3, 0xd1, 0xe3, 0x62, 0x9c, 0x6d, 0xa8, 0x3b, 0xe9, 0xce,
+	0xdc, 0xcc, 0x3b, 0xf3, 0x0e, 0xd8, 0x53, 0xc1, 0xe3, 0x5d, 0x9e, 0x30, 0xfd, 0xc8, 0x33, 0x58,
+	0x26, 0x45, 0xd9, 0x5a, 0x7a, 0x61, 0xa0, 0x13, 0x50, 0x0a, 0x3a, 0x7f, 0x2f, 0x41, 0x55, 0x3f,
+	0x74, 0x79, 0xe6, 0x34, 0x2b, 0x05, 0xe5, 0x67, 0x1e, 0x35, 0xcc, 0x62, 0xd4, 0x28, 0xce, 0x00,
+	0x4a, 0x4f, 0x37, 0x03, 0x20, 0xdf, 0x81, 0x46, 0xa4, 0xf6, 0x8a, 0xb1, 0xe6, 0x85, 0x45, 0x3a,
+	0xfd, 0x8b, 0xb4, 0xf5, 0x28, 0x07, 0xe4, 0x4b, 0xc1, 0x3e, 0x28, 0x61, 0x63, 0x34, 0x78, 0x83,
+	0x56, 0x25, 0x3c, 0x60, 0xe3, 0x27, 0x44, 0x9c, 0xa7, 0x08, 0x1a, 0xb2, 0xf4, 0x0d, 0xa3, 0x76,
+	0x03, 0x03, 0x81, 0x0c, 0x34, 0xc5, 0x18, 0xd0, 0x9c, 0x8f, 0x01, 0x2f, 0x40, 0x6d, 0x14, 0x4e,
+	0x26, 0x1e, 0xee, 0x2d, 0xab, 0x34, 0xad, 0x16, 0x06, 0xc2, 0xf9, 0x1a, 0xaa, 0x5a, 0x60, 0x52,
+	0x87, 0xea, 0x76, 0xf7, 0xc1, 0xe6, 0xe1, 0x8e, 0x8c, 0x42, 0x00, 0xd6, 0xfd, 0xde, 0xde, 0x26,
+	0xfd, 0xa2, 0x65, 0xc8, 0x88, 0xd4, 0xdb, 0x1b, 0xb4, 0x4c, 0x52, 0x83, 0xca, 0x83, 0x9d, 0xfd,
+	0xcd, 0x41, 0xab, 0x44, 0x6c, 0x28, 0xdf, 0xdf, 0xdf, 0xdf, 0x69, 0x95, 0x49, 0x03, 0xec, 0xed,
+	0xcd, 0x41, 0x77, 0xd0, 0xdb, 0xed, 0xb6, 0x2a, 0x12, 0xf7, 0x61, 0x77, 0xbf, 0x65, 0xc9, 0x8f,
+	0xc3, 0xde, 0x76, 0xab, 0x2a, 0xf7, 0x0f, 0x36, 0xfb, 0xfd, 0xcf, 0xf7, 0xe9, 0x76, 0xcb, 0x96,
+	0xe7, 0xf6, 0x07, 0xb4, 0xb7, 0xf7, 0xb0, 0x55, 0x73, 0xee, 0x42, 0xbd, 0xa0, 0x34, 0x49, 0x41,
+	0xbb, 0x0f, 0x5a, 0x4b, 0x92, 0xcd, 0xe3, 0xcd, 0x9d, 0xc3, 0x6e, 0xcb, 0x20, 0xcb, 0x00, 0xf8,
+	0x39, 0xdc, 0xd9, 0xdc, 0x7b, 0xd8, 0x32, 0x9d, 0xef, 0x1b, 0x19, 0x0d, 0xf6, 0xd6, 0x6f, 0x82,
+	0xad, 0x55, 0x9d, 0xd6, 0xce, 0xd7, 0x16, 0xec, 0x42, 0x33, 0x04, 0xe9, 0x66, 0xa3, 0x13, 0x3e,
+	0x3a, 0x15, 0xd3, 0x89, 0xf6, 0x8a, 0x0c, 0x56, 0x2d, 0xb2, 0xd4, 0x49, 0x1a, 0x04, 0x14, 0x94,
+	0xcd, 0x99, 0xca, 0x88, 0xaf, 0xe6, 0x4c, 0xf7, 0x00, 0xf2, 0x49, 0xc6, 0x25, 0x55, 0xef, 0x75,
+	0xa8, 0x30, 0xdf, 0x63, 0x42, 0x67, 0x31, 0x05, 0x38, 0x14, 0xea, 0x85, 0xf9, 0x87, 0x34, 0x18,
+	0xf3, 0xfd, 0xe1, 0x29, 0x3f, 0x17, 0x48, 0x6b, 0xd3, 0x2a, 0xf3, 0xfd, 0x47, 0xfc, 0x5c, 0x90,
+	0x35, 0xa8, 0xa8, 0xf1, 0x89, 0x79, 0x49, 0xa3, 0x8d, 0xe4, 0x54, 0x21, 0x38, 0x77, 0xc0, 0x52,
+	0xdd, 0x77, 0xc1, 0x67, 0x8c, 0x27, 0x26, 0x9a, 0x8f, 0xf5, 0xbd, 0xb1, 0x57, 0x27, 0x6f, 0xeb,
+	0x51, 0x8d, 0x50, 0x03, 0x22, 0x63, 0xbe, 0x0c, 0x53, 0x88, 0x7a, 0x4a, 0x83, 0x04, 0xce, 0x36,
+	0xd8, 0x57, 0x0e, 0xc2, 0xb4, 0x22, 0xcc, 0x5c, 0x11, 0x97, 0x8c, 0xc6, 0x9c, 0x18, 0x20, 0x1f,
+	0xe7, 0x68, 0x37, 0x56, 0xa7, 0x48, 0x37, 0x5e, 0x97, 0x26, 0xf2, 0x7c, 0x37, 0xe6, 0xc1, 0x05,
+	0xe9, 0xf3, 0x21, 0x50, 0x86, 0x43, 0x5e, 0x85, 0x32, 0x4e, 0xad, 0x4a, 0xf3, 0x01, 0x2b, 0x1b,
+	0x59, 0xe1, 0xae, 0x73, 0x04, 0x4d, 0x95, 0xc3, 0x28, 0xff, 0x6a, 0xca, 0xc5, 0x95, 0x95, 0xd2,
+	0x0a, 0x40, 0x16, 0x66, 0xd3, 0x39, 0x5c, 0x61, 0x45, 0x3a, 0xca, 0xb1, 0xc7, 0x7d, 0x37, 0x95,
+	0x4a, 0x43, 0xce, 0x07, 0xd0, 0x48, 0x79, 0x60, 0x9b, 0x7d, 0x3b, 0xcb, 0xa6, 0xa9, 0x5f, 0x4a,
+	0x83, 0x28, 0x94, 0xbd, 0xd0, 0xcd, 0x12, 0xa9, 0xf3, 0x67, 0x33, 0xa5, 0xd4, 0x4d, 0xe4, 0x5c,
+	0xad, 0x66, 0x2c, 0xd6, 0x6a, 0xf3, 0x75, 0x8f, 0xf9, 0xd4, 0x75, 0xcf, 0xff, 0x43, 0xcd, 0xc5,
+	0xa4, 0xef, 0x9d, 0xa5, 0xa1, 0x6f, 0xe5, 0xb2, 0x04, 0xaf, 0x4b, 0x03, 0xef, 0x8c, 0xd3, 0x9c,
+	0x40, 0xde, 0x29, 0x09, 0x4f, 0x79, 0xe0, 0x7d, 0x8d, 0xdd, 0xb2, 0x14, 0x3c, 0x5f, 0xc8, 0x07,
+	0x1a, 0xaa, 0x10, 0xd0, 0x03, 0x8d, 0x74, 0x98, 0x63, 0x15, 0x86, 0x39, 0x37, 0xc0, 0x9a, 0x46,
+	0x82, 0xc7, 0x49, 0x5a, 0x20, 0x2a, 0x28, 0x2b, 0xb2, 0x6a, 0x1a, 0x97, 0x05, 0x63, 0xe7, 0x7f,
+	0xa1, 0x96, 0xdd, 0x45, 0xc6, 0x9b, 0xbd, 0xfd, 0xbd, 0xae, 0x8a, 0x0e, 0xbd, 0xbd, 0xed, 0xee,
+	0x77, 0x5b, 0x86, 0x8c, 0x58, 0xb4, 0xfb, 0xb8, 0x4b, 0xfb, 0xdd, 0x96, 0x29, 0x23, 0xcb, 0x76,
+	0x77, 0xa7, 0x3b, 0xe8, 0xb6, 0x4a, 0x9f, 0x96, 0xed, 0x6a, 0xcb, 0xa6, 0x36, 0x9f, 0x45, 0xbe,
+	0x37, 0xf2, 0x12, 0xe7, 0x0b, 0xb0, 0x77, 0x59, 0x74, 0xa1, 0xf0, 0xcf, 0x13, 0xd2, 0x54, 0x4f,
+	0x0c, 0x74, 0xf2, 0x78, 0x1d, 0xaa, 0x3a, 0x6a, 0x68, 0xcf, 0xba, 0x10, 0x55, 0xd2, 0x7d, 0xe7,
+	0xe7, 0x06, 0x5c, 0xdf, 0x0d, 0xcf, 0x78, 0x96, 0x85, 0x0f, 0xd8, 0xb9, 0x1f, 0x32, 0xf7, 0x5b,
+	0xcc, 0x78, 0x0b, 0xae, 0x89, 0x70, 0x1a, 0x8f, 0xf8, 0x70, 0x61, 0x62, 0xd1, 0x54, 0xcb, 0x0f,
+	0xb5, 0x3b, 0x3a, 0xd0, 0x74, 0xb9, 0x48, 0x72, 0xac, 0x12, 0x62, 0xd5, 0xe5, 0x62, 0x8a, 0x93,
+	0x95, 0x13, 0xe5, 0xa7, 0x29, 0x27, 0x9c, 0xdf, 0x1a, 0xd0, 0xec, 0xce, 0xa2, 0x30, 0x4e, 0xd2,
+	0xab, 0x3e, 0x27, 0xcb, 0xf6, 0xaf, 0xd2, 0xc7, 0x50, 0xa6, 0x95, 0x98, 0x7f, 0xd5, 0xbb, 0x72,
+	0x9c, 0x72, 0x0f, 0x2c, 0x79, 0xd8, 0x54, 0x68, 0x57, 0x7a, 0x31, 0xe5, 0x39, 0x77, 0xf0, 0x7a,
+	0x1f, 0x71, 0xa8, 0xc6, 0x2d, 0x96, 0x5a, 0xe5, 0xb9, 0x52, 0xeb, 0x23, 0xb0, 0x14, 0x6a, 0xc1,
+	0xce, 0x75, 0xa8, 0xf6, 0x0f, 0xb7, 0xb6, 0xba, 0xfd, 0x7e, 0xcb, 0x20, 0x4d, 0xa8, 0x6d, 0x1f,
+	0x1e, 0xec, 0xf4, 0xb6, 0x36, 0x07, 0xda, 0xd6, 0x0f, 0x36, 0x7b, 0x3b, 0xdd, 0xed, 0x56, 0xc9,
+	0xd9, 0x82, 0xda, 0x60, 0x16, 0x68, 0xf2, 0x62, 0x1a, 0x34, 0xae, 0x48, 0x83, 0xe6, 0x42, 0x1a,
+	0xfc, 0x1c, 0xea, 0x85, 0x92, 0x88, 0xbc, 0x06, 0xe5, 0x64, 0x16, 0x5c, 0x9c, 0xf3, 0xa6, 0x7c,
+	0x28, 0x6e, 0x93, 0x97, 0xa1, 0x21, 0x9b, 0x33, 0x26, 0x84, 0x37, 0x0e, 0xb8, 0xab, 0x4f, 0x95,
+	0x0d, 0xdb, 0xa6, 0x5e, 0x72, 0x5e, 0x82, 0xa6, 0xec, 0x87, 0xbd, 0x09, 0x17, 0x09, 0x9b, 0x44,
+	0x98, 0xb8, 0x75, 0x90, 0x2e, 0x53, 0x33, 0x11, 0xce, 0x2d, 0x68, 0x1c, 0x70, 0x1e, 0x53, 0x2e,
+	0xa2, 0x30, 0x10, 0xd8, 0x19, 0x69, 0xcd, 0xaa, 0xac, 0xa0, 0x21, 0xe7, 0x08, 0x6a, 0xb2, 0xa8,
+	0xbd, 0xcf, 0x92, 0xd1, 0xc9, 0x3f, 0x5b, 0xf8, 0xde, 0x82, 0x6a, 0xa4, 0x2c, 0xa2, 0x4b, 0x57,
+	0x35, 0xc4, 0xd3, 0x56, 0xa2, 0xe9, 0xa6, 0x73, 0x13, 0x4a, 0x7b, 0xd3, 0x49, 0xf1, 0xaf, 0x93,
+	0x32, 0x16, 0x68, 0xce, 0x03, 0x68, 0xa4, 0x35, 0x20, 0x16, 0x65, 0x52, 0x97, 0xbe, 0xc7, 0x83,
+	0x82, 0x9e, 0x6d, 0xb5, 0x30, 0x10, 0x57, 0xb8, 0xcd, 0xc6, 0x6f, 0x0c, 0x28, 0xcb, 0x1b, 0xca,
+	0xb8, 0xdd, 0x1d, 0x9d, 0x84, 0x64, 0xee, 0x22, 0x9d, 0x39, 0xc8, 0x59, 0x22, 0xef, 0xa8, 0xa1,
+	0x62, 0x3a, 0x89, 0x7d, 0xa6, 0x28, 0x24, 0x2a, 0xe2, 0x02, 0xc5, 0x06, 0xd4, 0x3f, 0x0d, 0xbd,
+	0x60, 0x4b, 0xcd, 0xda, 0xc8, 0x65, 0x6a, 0xb9, 0x40, 0xf3, 0x1e, 0x58, 0x3d, 0x21, 0x6d, 0x70,
+	0x39, 0x7a, 0xd6, 0x06, 0x15, 0xcd, 0xe4, 0x2c, 0x6d, 0xfc, 0xb4, 0x04, 0xe5, 0x2f, 0x79, 0x1c,
+	0x92, 0x7b, 0x50, 0xd5, 0xc3, 0x00, 0xb2, 0xd0, 0xf4, 0x77, 0xb2, 0xa7, 0xb8, 0x30, 0x2d, 0x70,
+	0x96, 0xc8, 0xfb, 0x60, 0xe9, 0x78, 0x3f, 0x3f, 0xb1, 0xe8, 0x3c, 0xe9, 0xf9, 0x3a, 0x4b, 0x6b,
+	0xc6, 0x3b, 0x06, 0x79, 0x1b, 0x2c, 0xe5, 0xa9, 0x0b, 0xba, 0xbb, 0xac, 0xb4, 0x77, 0x96, 0x90,
+	0xa0, 0xde, 0x3f, 0x09, 0xa7, 0xbe, 0xdb, 0xe7, 0xf1, 0x19, 0x27, 0x0b, 0xf3, 0xb1, 0xce, 0x02,
+	0xec, 0x2c, 0x91, 0xb7, 0x00, 0x94, 0xfb, 0x1e, 0x7a, 0xae, 0x20, 0xf5, 0x74, 0x7f, 0x6f, 0x3a,
+	0xe9, 0xb4, 0x90, 0x65, 0xea, 0xdc, 0x3d, 0x57, 0x28, 0xf4, 0x82, 0x7b, 0x7f, 0x2b, 0xfa, 0xbb,
+	0xd0, 0xdc, 0xc2, 0x57, 0xb7, 0x1f, 0x6f, 0x1e, 0x85, 0x71, 0x42, 0x16, 0x27, 0x66, 0x9d, 0xc5,
+	0x05, 0x67, 0x89, 0x7c, 0x08, 0xf6, 0x20, 0x3e, 0x57, 0xf8, 0xcf, 0x15, 0x5e, 0x63, 0xce, 0xf8,
+	0x09, 0xf2, 0x6f, 0xfc, 0xb1, 0x04, 0xd6, 0xe7, 0x61, 0x7c, 0xca, 0x63, 0xb2, 0x0e, 0x16, 0x76,
+	0x6a, 0x05, 0x47, 0xca, 0x3a, 0xb7, 0xcb, 0x98, 0xde, 0x81, 0x1a, 0xaa, 0x6c, 0xc0, 0xc4, 0x69,
+	0x6e, 0x24, 0xfc, 0xf3, 0x2e, 0xd7, 0x9a, 0xca, 0xf6, 0x28, 0xd7, 0x72, 0x3f, 0x89, 0x39, 0x9b,
+	0x64, 0xdd, 0xe9, 0x85, 0xf6, 0xa9, 0x53, 0xcf, 0x3b, 0xa4, 0x3e, 0xda, 0xe6, 0x2e, 0x94, 0xfb,
+	0x52, 0xa6, 0xfc, 0xef, 0xb6, 0xfc, 0x8f, 0x87, 0x0e, 0x29, 0x2e, 0x66, 0x7c, 0x3e, 0x00, 0x4b,
+	0xa5, 0xeb, 0x5c, 0x11, 0x73, 0xb5, 0x4d, 0xe7, 0xfa, 0xe2, 0xb2, 0x26, 0xbc, 0x05, 0xd5, 0x83,
+	0x69, 0x3c, 0xe6, 0x03, 0xb1, 0xe0, 0x39, 0x45, 0x93, 0xa1, 0xae, 0x2d, 0x15, 0xc4, 0x73, 0x06,
+	0x73, 0x41, 0xbd, 0x73, 0xf9, 0xb2, 0xb3, 0x44, 0xee, 0x42, 0x8b, 0xf2, 0x11, 0xf7, 0x0a, 0xc9,
+	0x90, 0x14, 0x45, 0x5e, 0x7c, 0x79, 0x6b, 0x06, 0xf9, 0x18, 0x9a, 0x73, 0xc9, 0x93, 0x64, 0x89,
+	0xe4, 0xb2, 0x9c, 0xba, 0x78, 0xc0, 0xfd, 0xd6, 0xaf, 0xbf, 0x59, 0x31, 0x7e, 0xff, 0xcd, 0x8a,
+	0xf1, 0xa7, 0x6f, 0x56, 0x8c, 0x1f, 0xfd, 0x65, 0x65, 0xe9, 0xc8, 0xc2, 0xbf, 0x89, 0xdf, 0xfd,
+	0x47, 0x00, 0x00, 0x00, 0xff, 0xff, 0x89, 0x65, 0x8d, 0x0a, 0x4b, 0x1e, 0x00, 0x00,
 }
diff --git a/protos/internal.proto b/protos/internal.proto
index 36097188..4377221d 100644
--- a/protos/internal.proto
+++ b/protos/internal.proto
@@ -396,7 +396,7 @@ service Zero {
 	rpc AssignUids (Num)               returns (api.AssignedIds) {}
 	rpc Timestamps (Num)               returns (api.AssignedIds) {}
 	rpc CommitOrAbort (api.TxnContext) returns (api.TxnContext) {}
-	rpc TryAbort (TxnTimestamps)       returns (TxnTimestamps) {}
+	rpc TryAbort (TxnTimestamps)       returns (OracleDelta) {}
 }
 
 service Worker {
diff --git a/query/common_test.go b/query/common_test.go
index 7b12b73f..6b065d2b 100644
--- a/query/common_test.go
+++ b/query/common_test.go
@@ -66,7 +66,7 @@ func addEdge(t *testing.T, attr string, src uint64, edge *intern.DirectedEdge) {
 		l.AddMutationWithIndex(context.Background(), edge, txn))
 
 	commit := timestamp()
-	require.NoError(t, txn.CommitMutations(context.Background(), commit))
+	require.NoError(t, txn.CommitToMemory(commit))
 	delta := &intern.OracleDelta{MaxAssigned: commit}
 	delta.Txns = append(delta.Txns, &intern.TxnStatus{StartTs: startTs, CommitTs: commit})
 	posting.Oracle().ProcessDelta(delta)
diff --git a/worker/draft.go b/worker/draft.go
index 0a3cf0b9..2368480b 100644
--- a/worker/draft.go
+++ b/worker/draft.go
@@ -416,29 +416,44 @@ func (n *node) processApplyCh() {
 }
 
 func (n *node) commitOrAbort(pkey string, delta *intern.OracleDelta) error {
-	ctx := n.Ctx(pkey)
+	// First let's commit all mutations to disk.
+	writer := x.TxnWriter{DB: pstore}
+	toDisk := func(start, commit uint64) {
+		txn := posting.Oracle().GetTxn(start)
+		if txn == nil {
+			return
+		}
+		for err := txn.CommitToDisk(&writer, commit); err != nil; {
+			x.Printf("Error while applying txn status to disk (%d -> %d): %v",
+				start, commit, err)
+			time.Sleep(10 * time.Millisecond)
+		}
+	}
+	for _, status := range delta.Txns {
+		toDisk(status.StartTs, status.CommitTs)
+	}
+	if err := writer.Flush(); err != nil {
+		x.Errorf("Error while flushing to disk: %v", err)
+		return err
+	}
 
-	applyTxnStatus := func(startTs, commitTs uint64) {
-		var err error
-		for i := 0; i < 3; i++ {
-			err = commitOrAbort(ctx, startTs, commitTs)
-			if err == nil || err == posting.ErrInvalidTxn {
-				break
-			}
-			x.Printf("Error while applying txn status (%d -> %d): %v", startTs, commitTs, err)
+	// Now let's commit all mutations to memory.
+	toMemory := func(start, commit uint64) {
+		txn := posting.Oracle().GetTxn(start)
+		if txn == nil {
+			return
 		}
-		// TODO: Even after multiple tries, if we're unable to apply the status of a transaction,
-		// what should we do? Maybe do a printf, and let them know that there might be a disk issue.
-		if tr, ok := trace.FromContext(ctx); ok {
-			tr.LazyPrintf("Status of commitOrAbort startTs %d: %v\n", startTs, err)
+		for err := txn.CommitToMemory(commit); err != nil; {
+			x.Printf("Error while applying txn status to memory (%d -> %d): %v",
+				start, commit, err)
+			time.Sleep(10 * time.Millisecond)
 		}
 	}
 
 	for _, txn := range delta.Txns {
-		applyTxnStatus(txn.StartTs, txn.CommitTs)
+		toMemory(txn.StartTs, txn.CommitTs)
 	}
-	// TODO: Use MaxPending to track the txn watermark. That's the only thing we need really.
-	// delta.GetMaxPending
+	// Now advance Oracle(), so we can service waiting reads.
 	posting.Oracle().ProcessDelta(delta)
 	return nil
 }
@@ -697,29 +712,44 @@ func (n *node) Stop() {
 	<-n.done // wait for Run to respond.
 }
 
+var errConnection = errors.New("No connection exists")
+
+func (n *node) blockingAbort(req *intern.TxnTimestamps) error {
+	pl := groups().Leader(0)
+	if pl == nil {
+		return errConnection
+	}
+	zc := intern.NewZeroClient(pl.Get())
+	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
+	defer cancel()
+
+	delta, err := zc.TryAbort(ctx, req)
+	x.Printf("Aborted txns with start ts: %v. Error: %v\n", req.Ts, err)
+	if err != nil || len(delta.Txns) == 0 {
+		return err
+	}
+
+	// Let's propose the txn updates received from Zero. This is important because there are edge
+	// cases where a txn status might have been missed by the group.
+	n.elog.Printf("Proposing abort txn delta: %+v\n", delta)
+	proposal := &intern.Proposal{Delta: delta}
+	return n.proposeAndWait(n.ctx, proposal)
+}
+
 // abortOldTransactions would find txns which have done pre-writes, but have been pending for a
 // while. The time that is used is based on the last pre-write seen, so if a txn is doing a
 // pre-write multiple times, we'll pick the timestamp of the last pre-write. Thus, this function
 // would only act on the txns which have not been active in the last N minutes, and send them for
 // abort. Note that only the leader runs this function.
-// NOTE: We might need to get the results of TryAbort and propose them. But, it's unclear if we need
-// to, because Zero should stream out the aborts anyway.
 func (n *node) abortOldTransactions() {
-	pl := groups().Leader(0)
-	if pl == nil {
-		return
-	}
-	zc := intern.NewZeroClient(pl.Get())
 	// Aborts if not already committed.
 	startTimestamps := posting.Oracle().TxnOlderThan(5 * time.Minute)
 	if len(startTimestamps) == 0 {
 		return
 	}
 	req := &intern.TxnTimestamps{Ts: startTimestamps}
-	ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
-	defer cancel()
-	_, err := zc.TryAbort(ctx, req)
-	x.Printf("Aborted txns with start ts: %v. Error: %v\n", startTimestamps, err)
+	err := n.blockingAbort(req)
+	x.Printf("abortOldTransactions for %d txns. Error: %+v\n", len(req.Ts), err)
 }
 
 // calculateSnapshot would calculate a snapshot index, considering these factors:
diff --git a/worker/mutation.go b/worker/mutation.go
index e77af843..2be273f8 100644
--- a/worker/mutation.go
+++ b/worker/mutation.go
@@ -491,13 +491,10 @@ func populateMutationMap(src *intern.Mutations) map[uint32]*intern.Mutations {
 func commitOrAbort(ctx context.Context, startTs, commitTs uint64) error {
 	txn := posting.Oracle().GetTxn(startTs)
 	if txn == nil {
-		return posting.ErrInvalidTxn
+		return nil
 	}
 	// Ensures that we wait till prewrite is applied
-	if commitTs == 0 {
-		return txn.AbortMutations(ctx)
-	}
-	return txn.CommitMutations(ctx, commitTs)
+	return txn.CommitToMemory(commitTs)
 }
 
 type res struct {
@@ -588,32 +585,9 @@ func (w *grpcWorker) Mutate(ctx context.Context, m *intern.Mutations) (*api.TxnC
 }
 
 func tryAbortTransactions(startTimestamps []uint64) {
-	pl := groups().Leader(0)
-	if pl == nil {
-		return
-	}
-	zc := intern.NewZeroClient(pl.Get())
 	// Aborts if not already committed.
 	req := &intern.TxnTimestamps{Ts: startTimestamps}
-	resp, err := zc.TryAbort(context.Background(), req)
-	for err != nil {
-		x.Printf("Error while trying to abort txns: %v\n", err)
-		resp, err = zc.TryAbort(context.Background(), req)
-	}
-	commitTimestamps := resp.Ts
-	x.AssertTrue(len(startTimestamps) == len(commitTimestamps))
-
-	for i, startTs := range startTimestamps {
-		tctx := &api.TxnContext{StartTs: startTs, CommitTs: commitTimestamps[i]}
-		err := commitOrAbort(context.Background(), tctx.StartTs, tctx.CommitTs)
-		// Transaction could already have been aborted in which case it would be deleted from the
-		// transactions map and we should just continue.
-		// TODO - Make sure all other errors are transient, we don't want to be stuck in an infinite
-		// loop.
-		for err != nil && err != posting.ErrInvalidTxn {
-			x.Printf("Error while locally aborting txns: %v\n", err)
-			// This will fail only due to badger error.
-			err = commitOrAbort(context.Background(), tctx.StartTs, tctx.CommitTs)
-		}
-	}
+
+	err := groups().Node.blockingAbort(req)
+	x.Printf("tryAbortTransactions for %d txns. Error: %+v\n", len(req.Ts), err)
 }
diff --git a/worker/predicate_test.go b/worker/predicate_test.go
index cd4afbfb..c4ad9a3b 100644
--- a/worker/predicate_test.go
+++ b/worker/predicate_test.go
@@ -61,7 +61,11 @@ func commitTransaction(t *testing.T, edge *intern.DirectedEdge, l *posting.List)
 	require.NoError(t, err)
 
 	commit := commitTs(startTs)
-	require.NoError(t, txn.CommitMutations(context.Background(), commit))
+
+	writer := &x.TxnWriter{DB: pstore}
+	require.NoError(t, txn.CommitToDisk(writer, commit))
+	require.NoError(t, writer.Flush())
+	require.NoError(t, txn.CommitToMemory(commit))
 }
 
 // Hacky tests change laster
diff --git a/x/badger.go b/x/badger.go
new file mode 100644
index 00000000..87903d68
--- /dev/null
+++ b/x/badger.go
@@ -0,0 +1,48 @@
+package x
+
+import (
+	"math"
+	"sync"
+
+	"github.com/dgraph-io/badger"
+)
+
+type TxnWriter struct {
+	DB  *badger.ManagedDB
+	wg  sync.WaitGroup
+	che chan error
+}
+
+func (w *TxnWriter) cb(err error) {
+	defer w.wg.Done()
+	if err == nil {
+		return
+	}
+	select {
+	case w.che <- err:
+	default:
+	}
+}
+
+func (w *TxnWriter) SetAt(key, val []byte, meta byte, ts uint64) error {
+	txn := w.DB.NewTransactionAt(math.MaxUint64, true)
+	defer txn.Discard()
+	// TODO: We should probably do a Get to ensure that we don't end up
+	// overwriting an already existing value at that ts, which might be there
+	// due to a previous rollup event.
+	if err := txn.SetWithMeta(key, val, meta); err != nil {
+		return err
+	}
+	w.wg.Add(1)
+	return txn.CommitAt(ts, w.cb)
+}
+
+func (w *TxnWriter) Flush() error {
+	w.wg.Wait()
+	select {
+	case err := <-w.che:
+		return err
+	default:
+		return nil
+	}
+}
-- 
GitLab