made ValidatorSet.Hash deterministic; fix off-by-1 bugs

This commit is contained in:
Jae Kwon
2014-11-05 03:08:29 -08:00
parent a0cc186c75
commit 8718bd52a4
13 changed files with 207 additions and 100 deletions

View File

@ -102,7 +102,7 @@ func (psh PartSetHeader) WriteTo(w io.Writer) (n int64, err error) {
} }
func (psh PartSetHeader) String() string { func (psh PartSetHeader) String() string {
return fmt.Sprintf("PartSet{%X/%v}", psh.Hash, psh.Total) return fmt.Sprintf("PartSet{T:%v %X}", psh.Total, Fingerprint(psh.Hash))
} }
func (psh PartSetHeader) IsZero() bool { func (psh PartSetHeader) IsZero() bool {
@ -143,13 +143,13 @@ func NewPartSetFromData(data []byte) *PartSet {
partsBitArray.SetIndex(uint(i), true) partsBitArray.SetIndex(uint(i), true)
} }
// Compute merkle trails // Compute merkle trails
hashTree := merkle.HashTreeFromHashables(parts_) trails, rootTrail := merkle.HashTrailsFromHashables(parts_)
for i := 0; i < total; i++ { for i := 0; i < total; i++ {
parts[i].Trail = merkle.HashTrailForIndex(hashTree, i) parts[i].Trail = trails[i].Flatten()
} }
return &PartSet{ return &PartSet{
total: uint16(total), total: uint16(total),
hash: hashTree[len(hashTree)/2], hash: rootTrail.Hash,
parts: parts, parts: parts,
partsBitArray: partsBitArray, partsBitArray: partsBitArray,
count: uint16(total), count: uint16(total),
@ -236,7 +236,7 @@ func (ps *PartSet) AddPart(part *Part) (bool, error) {
} }
// Check hash trail // Check hash trail
if !merkle.VerifyHashTrailForIndex(int(part.Index), part.Hash(), part.Trail, ps.hash) { if !merkle.VerifyHashTrail(uint(part.Index), uint(ps.total), part.Hash(), part.Trail, ps.hash) {
return false, ErrPartSetInvalidTrail return false, ErrPartSetInvalidTrail
} }

View File

@ -5,6 +5,7 @@ import (
"io" "io"
. "github.com/tendermint/tendermint/binary" . "github.com/tendermint/tendermint/binary"
. "github.com/tendermint/tendermint/common"
) )
type Signable interface { type Signable interface {
@ -38,7 +39,7 @@ func (sig Signature) WriteTo(w io.Writer) (n int64, err error) {
} }
func (sig Signature) String() string { func (sig Signature) String() string {
return fmt.Sprintf("Signature{%v:%X}", sig.SignerId, sig.Bytes) return fmt.Sprintf("Signature{Id:%v %X}", sig.SignerId, Fingerprint(sig.Bytes))
} }
//------------------------------------- //-------------------------------------
@ -85,6 +86,10 @@ func (rsig RoundSignature) IsZero() bool {
return rsig.Round == 0 && rsig.SignerId == 0 && len(rsig.Bytes) == 0 return rsig.Round == 0 && rsig.SignerId == 0 && len(rsig.Bytes) == 0
} }
func (rsig RoundSignature) String() string {
return fmt.Sprintf("RoundSignature{R:%v Id:%v %X}", rsig.Round, rsig.SignerId, Fingerprint(rsig.Bytes))
}
//------------------------------------- //-------------------------------------
func ReadRoundSignatures(r io.Reader, n *int64, err *error) (rsigs []RoundSignature) { func ReadRoundSignatures(r io.Reader, n *int64, err *error) (rsigs []RoundSignature) {

View File

@ -116,6 +116,8 @@ func (bs *BlockStore) SaveBlock(block *Block, blockParts *PartSet) {
bs.db.Set(calcBlockValidationKey(height), validationBytes) bs.db.Set(calcBlockValidationKey(height), validationBytes)
// Save new BlockStoreJSON descriptor // Save new BlockStoreJSON descriptor
BlockStoreJSON{Height: height}.Save(bs.db) BlockStoreJSON{Height: height}.Save(bs.db)
// Done!
bs.height = height
} }
func (bs *BlockStore) saveBlockPart(height uint32, index uint16, part *Part) { func (bs *BlockStore) saveBlockPart(height uint32, index uint16, part *Part) {

View File

@ -80,5 +80,5 @@ func (v *Vote) String() string {
panic("Unknown vote type") panic("Unknown vote type")
} }
return fmt.Sprintf("%v{%v/%v:%X:%v:%v}", typeString, v.Height, v.Round, Fingerprint(v.BlockHash), v.BlockParts, v.SignerId) return fmt.Sprintf("%v{%v/%v %X#%v %v}", typeString, v.Height, v.Round, Fingerprint(v.BlockHash), v.BlockParts, v.Signature)
} }

View File

@ -744,7 +744,7 @@ func (m *PartMessage) WriteTo(w io.Writer) (n int64, err error) {
} }
func (m *PartMessage) String() string { func (m *PartMessage) String() string {
return fmt.Sprintf("[Part %v/%v T:%X]", m.Height, m.Round, m.Type) return fmt.Sprintf("[Part %v/%v T:%X %v]", m.Height, m.Round, m.Type, m.Part)
} }
//------------------------------------- //-------------------------------------

View File

@ -529,7 +529,7 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
validation = cs.LastCommits.MakeValidation() validation = cs.LastCommits.MakeValidation()
} }
} }
txs, state := cs.mempool.GetProposalTxs() // TODO: cache state txs := cs.mempool.GetProposalTxs()
block = &Block{ block = &Block{
Header: &Header{ Header: &Header{
Network: Config.Network, Network: Config.Network,
@ -538,13 +538,18 @@ func (cs *ConsensusState) RunActionPropose(height uint32, round uint16) {
Fees: 0, // TODO fees Fees: 0, // TODO fees
LastBlockHash: cs.state.LastBlockHash, LastBlockHash: cs.state.LastBlockHash,
LastBlockParts: cs.state.LastBlockParts, LastBlockParts: cs.state.LastBlockParts,
StateHash: state.Hash(), StateHash: nil, // Will set afterwards.
}, },
Validation: validation, Validation: validation,
Data: &Data{ Data: &Data{
Txs: txs, Txs: txs,
}, },
} }
// Set the block.Header.StateHash.
// TODO: we could cache the resulting state to cs.stagedState.
cs.state.Copy().AppendBlock(block, PartSetHeader{}, false)
blockParts = NewPartSetFromData(BinaryBytes(block)) blockParts = NewPartSetFromData(BinaryBytes(block))
pol = cs.LockedPOL // If exists, is a PoUnlock. pol = cs.LockedPOL // If exists, is a PoUnlock.
} }
@ -586,6 +591,7 @@ func (cs *ConsensusState) RunActionPrevote(height uint32, round uint16) {
// If ProposalBlock is nil, prevote nil. // If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil { if cs.ProposalBlock == nil {
log.Warning("ProposalBlock is nil")
cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{}) cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{})
return return
} }
@ -594,6 +600,7 @@ func (cs *ConsensusState) RunActionPrevote(height uint32, round uint16) {
err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts) err := cs.stageBlock(cs.ProposalBlock, cs.ProposalBlockParts)
if err != nil { if err != nil {
// ProposalBlock is invalid, prevote nil. // ProposalBlock is invalid, prevote nil.
log.Warning("ProposalBlock is invalid: %v", err)
cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{}) cs.signAddVote(VoteTypePrevote, nil, PartSetHeader{})
return return
} }

View File

@ -41,10 +41,10 @@ func (mem *Mempool) AddTx(tx Tx) (err error) {
} }
} }
func (mem *Mempool) GetProposalTxs() ([]Tx, *state.State) { func (mem *Mempool) GetProposalTxs() []Tx {
mem.mtx.Lock() mem.mtx.Lock()
defer mem.mtx.Unlock() defer mem.mtx.Unlock()
return mem.txs, mem.state return mem.txs
} }
// "block" is the new block being committed. // "block" is the new block being committed.

View File

@ -1,23 +1,24 @@
/* /*
Computes a deterministic minimal height merkle tree hash. Computes a deterministic minimal height merkle tree hash.
If the number of items is not a power of two, some leaves If the number of items is not a power of two, some leaves
will be at different levels, leaning left. will be at different levels. Tries to keep both sides of
the tree the same size, but the left may be one greater.
Use this for shorter deterministic trees, such as the validator list. Use this for short deterministic trees, such as the validator list.
For larger datasets, use IAVLTree. For larger datasets, use IAVLTree.
* *
/ \ / \
/ \ / \
/ \ / \
/ \ / \
* * * *
/ \ / \ / \ / \
/ \ / \ / \ / \
/ \ / \ / \ / \
* * * h6 * * * h6
/ \ / \ / \ / \ / \ / \
h0 h1 h2 h3 h4 h5 h0 h1 h2 h3 h4 h5
*/ */
@ -46,7 +47,7 @@ func HashFromHashes(hashes [][]byte) []byte {
// Recursive impl. // Recursive impl.
switch len(hashes) { switch len(hashes) {
case 0: case 0:
panic("Cannot call HashFromHashes() with 0 length arg") return nil
case 1: case 1:
return hashes[0] return hashes[0]
default: default:
@ -110,7 +111,7 @@ func HashTrailsFromHashables(items []Hashable) (trails []*HashTrail, root *HashT
// Recursive impl. // Recursive impl.
switch len(items) { switch len(items) {
case 0: case 0:
panic("Cannot call HashTrailsFromHashables() with 0 length arg") return nil, nil
case 1: case 1:
trail := &HashTrail{items[0].Hash(), nil, nil, nil} trail := &HashTrail{items[0].Hash(), nil, nil, nil}
return []*HashTrail{trail}, trail return []*HashTrail{trail}, trail

View File

@ -172,8 +172,10 @@ func (sw *Switch) Broadcast(chId byte, msg Binary) (numSuccess, numFailure int)
log.Debug("[%X] Broadcast: %v", chId, msg) log.Debug("[%X] Broadcast: %v", chId, msg)
for _, peer := range sw.peers.List() { for _, peer := range sw.peers.List() {
success := peer.TrySend(chId, msg) // XXX XXX Change.
// log.Debug("Broadcast for peer %v success: %v", peer, success) // success := peer.TrySend(chId, msg)
success := peer.Send(chId, msg)
log.Debug("[%X] for peer %v success: %v", chId, peer, success)
if success { if success {
numSuccess += 1 numSuccess += 1
} else { } else {

View File

@ -317,7 +317,7 @@ func (s *State) AppendBlock(block *Block, blockPartsHeader PartSetHeader, checkS
return true return true
} }
vote := &Vote{ vote := &Vote{
Height: block.Height, Height: block.Height - 1,
Round: rsig.Round, Round: rsig.Round,
Type: VoteTypeCommit, Type: VoteTypeCommit,
BlockHash: block.LastBlockHash, BlockHash: block.LastBlockHash,
@ -328,6 +328,7 @@ func (s *State) AppendBlock(block *Block, blockPartsHeader PartSetHeader, checkS
sumVotingPower += val.VotingPower sumVotingPower += val.VotingPower
return false return false
} else { } else {
log.Warning("Invalid validation signature.\nval: %v\nvote: %v", val, vote)
err = errors.New("Invalid validation signature") err = errors.New("Invalid validation signature")
return true return true
} }
@ -355,7 +356,7 @@ func (s *State) AppendBlock(block *Block, blockPartsHeader PartSetHeader, checkS
if val == nil { if val == nil {
return ErrStateInvalidSignature return ErrStateInvalidSignature
} }
val.LastCommitHeight = block.Height val.LastCommitHeight = block.Height - 1
updated := s.BondedValidators.Update(val) updated := s.BondedValidators.Update(val)
if !updated { if !updated {
panic("Failed to update validator LastCommitHeight") panic("Failed to update validator LastCommitHeight")

View File

@ -145,11 +145,8 @@ func TestGenesisSaveLoad(t *testing.T) {
if s0.BondedValidators.TotalVotingPower() != s1.BondedValidators.TotalVotingPower() { if s0.BondedValidators.TotalVotingPower() != s1.BondedValidators.TotalVotingPower() {
t.Error("BondedValidators TotalVotingPower mismatch") t.Error("BondedValidators TotalVotingPower mismatch")
} }
if bytes.Equal(s0.BondedValidators.Hash(), s1.BondedValidators.Hash()) { if !bytes.Equal(s0.BondedValidators.Hash(), s1.BondedValidators.Hash()) {
// The BondedValidators hash should have changed because t.Error("BondedValidators hash mismatch")
// each AppendBlock() calls IncrementAccum(),
// changing each validator's Accum.
t.Error("BondedValidators hash should have changed")
} }
if s0.UnbondingValidators.Size() != s1.UnbondingValidators.Size() { if s0.UnbondingValidators.Size() != s1.UnbondingValidators.Size() {
t.Error("UnbondingValidators Size mismatch") t.Error("UnbondingValidators Size mismatch")

View File

@ -33,14 +33,8 @@ func ReadValidator(r io.Reader, n *int64, err *error) *Validator {
// Creates a new copy of the validator so we can mutate accum. // Creates a new copy of the validator so we can mutate accum.
func (v *Validator) Copy() *Validator { func (v *Validator) Copy() *Validator {
return &Validator{ vCopy := *v
Account: v.Account, return &vCopy
BondHeight: v.BondHeight,
UnbondHeight: v.UnbondHeight,
LastCommitHeight: v.LastCommitHeight,
VotingPower: v.VotingPower,
Accum: v.Accum,
}
} }
// Used to persist the state of ConsensusStateControl. // Used to persist the state of ConsensusStateControl.
@ -75,7 +69,17 @@ func (v *Validator) CompareAccum(other *Validator) *Validator {
} }
func (v *Validator) String() string { func (v *Validator) String() string {
return fmt.Sprintf("Validator{%v VP:%v A:%v}", v.Account, v.VotingPower, v.Accum) return fmt.Sprintf("Validator{%v %v-%v-%v VP:%v A:%v}",
v.Account,
v.BondHeight,
v.LastCommitHeight,
v.UnbondHeight,
v.VotingPower,
v.Accum)
}
func (v *Validator) Hash() []byte {
return BinaryHash(v)
} }
//------------------------------------- //-------------------------------------

View File

@ -3,50 +3,71 @@ package state
import ( import (
"fmt" "fmt"
"io" "io"
"sort"
"strings" "strings"
. "github.com/tendermint/tendermint/binary" . "github.com/tendermint/tendermint/binary"
"github.com/tendermint/tendermint/merkle" "github.com/tendermint/tendermint/merkle"
) )
//-------------------------------------
// Implements sort for sorting validators by id.
type ValidatorSlice []*Validator
func (vs ValidatorSlice) Len() int {
return len(vs)
}
func (vs ValidatorSlice) Less(i, j int) bool {
return vs[i].Id < vs[j].Id
}
func (vs ValidatorSlice) Swap(i, j int) {
it := vs[i]
vs[i] = vs[j]
vs[j] = it
}
//-------------------------------------
// Not goroutine-safe. // Not goroutine-safe.
// TODO: consider validator Accum overflow? // TODO: consider validator Accum overflow?
// TODO: replace validators []*Validator with github.com/jaekwon/go-ibbs?
// NOTE: all get/set to validators should copy the value for safety.
type ValidatorSet struct { type ValidatorSet struct {
validators merkle.Tree validators []*Validator
proposer *Validator // Whoever has the highest Accum.
// cache
proposer *Validator
totalVotingPower uint64 totalVotingPower uint64
} }
func NewValidatorSet(vals []*Validator) *ValidatorSet { func NewValidatorSet(vals []*Validator) *ValidatorSet {
validators := merkle.NewIAVLTree(BasicCodec, ValidatorCodec, 0, nil) // In memory validators := make([]*Validator, len(vals))
var proposer *Validator for i, val := range vals {
totalVotingPower := uint64(0) validators[i] = val.Copy()
for _, val := range vals {
validators.Set(val.Id, val)
proposer = proposer.CompareAccum(val)
totalVotingPower += val.VotingPower
} }
sort.Sort(ValidatorSlice(validators))
return &ValidatorSet{ return &ValidatorSet{
validators: validators, validators: validators,
proposer: proposer,
totalVotingPower: totalVotingPower,
} }
} }
func ReadValidatorSet(r io.Reader, n *int64, err *error) *ValidatorSet { func ReadValidatorSet(r io.Reader, n *int64, err *error) *ValidatorSet {
size := ReadUInt64(r, n, err) size := ReadUVarInt(r, n, err)
validators := []*Validator{} validators := []*Validator{}
for i := uint64(0); i < size; i++ { for i := uint(0); i < size; i++ {
validator := ReadValidator(r, n, err) validator := ReadValidator(r, n, err)
validators = append(validators, validator) validators = append(validators, validator)
} }
sort.Sort(ValidatorSlice(validators))
return NewValidatorSet(validators) return NewValidatorSet(validators)
} }
func (vset *ValidatorSet) WriteTo(w io.Writer) (n int64, err error) { func (vset *ValidatorSet) WriteTo(w io.Writer) (n int64, err error) {
WriteUInt64(w, uint64(vset.validators.Size()), &n, &err) WriteUVarInt(w, uint(len(vset.validators)), &n, &err)
vset.validators.Iterate(func(key_ interface{}, val_ interface{}) bool { vset.Iterate(func(index uint, val *Validator) bool {
val := val_.(*Validator)
WriteBinary(w, val, &n, &err) WriteBinary(w, val, &n, &err)
return false return false
}) })
@ -55,85 +76,152 @@ func (vset *ValidatorSet) WriteTo(w io.Writer) (n int64, err error) {
func (vset *ValidatorSet) IncrementAccum() { func (vset *ValidatorSet) IncrementAccum() {
// Decrement from previous proposer // Decrement from previous proposer
vset.proposer.Accum -= int64(vset.totalVotingPower) oldProposer := vset.Proposer()
var proposer *Validator oldProposer.Accum -= int64(vset.TotalVotingPower())
// Increment accum and find proposer vset.Update(oldProposer)
vset.validators.Iterate(func(key_ interface{}, val_ interface{}) bool { var newProposer *Validator
val := val_.(*Validator) // Increment accum and find new proposer
// NOTE: updates validators in place.
for _, val := range vset.validators {
val.Accum += int64(val.VotingPower) val.Accum += int64(val.VotingPower)
proposer = proposer.CompareAccum(val) newProposer = newProposer.CompareAccum(val)
return false }
}) vset.proposer = newProposer
vset.proposer = proposer
} }
func (vset *ValidatorSet) Copy() *ValidatorSet { func (vset *ValidatorSet) Copy() *ValidatorSet {
validators := make([]*Validator, len(vset.validators))
for i, val := range vset.validators {
// NOTE: must copy, since IncrementAccum updates in place.
validators[i] = val.Copy()
}
return &ValidatorSet{ return &ValidatorSet{
validators: vset.validators.Copy(), validators: validators,
proposer: vset.proposer, proposer: vset.proposer,
totalVotingPower: vset.totalVotingPower, totalVotingPower: vset.totalVotingPower,
} }
} }
func (vset *ValidatorSet) GetById(id uint64) (index uint, val *Validator) { func (vset *ValidatorSet) HasId(id uint64) bool {
index_, val_ := vset.validators.Get(id) idx := sort.Search(len(vset.validators), func(i int) bool {
index, val = uint(index_), val_.(*Validator) return id <= vset.validators[i].Id
return })
return idx != len(vset.validators) && vset.validators[idx].Id == id
} }
func (vset *ValidatorSet) HasId(id uint64) bool { func (vset *ValidatorSet) GetById(id uint64) (index uint, val *Validator) {
_, val_ := vset.validators.Get(id) idx := sort.Search(len(vset.validators), func(i int) bool {
return val_ != nil return id <= vset.validators[i].Id
})
if idx != len(vset.validators) && vset.validators[idx].Id == id {
return uint(idx), vset.validators[idx].Copy()
} else {
return 0, nil
}
} }
func (vset *ValidatorSet) GetByIndex(index uint) (id uint64, val *Validator) { func (vset *ValidatorSet) GetByIndex(index uint) (id uint64, val *Validator) {
id_, val_ := vset.validators.GetByIndex(uint64(index)) val = vset.validators[index]
id, val = id_.(uint64), val_.(*Validator) return val.Id, val.Copy()
return
} }
func (vset *ValidatorSet) Size() uint { func (vset *ValidatorSet) Size() uint {
return uint(vset.validators.Size()) return uint(len(vset.validators))
} }
func (vset *ValidatorSet) TotalVotingPower() uint64 { func (vset *ValidatorSet) TotalVotingPower() uint64 {
if vset.totalVotingPower == 0 {
for _, val := range vset.validators {
vset.totalVotingPower += val.VotingPower
}
}
return vset.totalVotingPower return vset.totalVotingPower
} }
func (vset *ValidatorSet) Proposer() (proposer *Validator) { func (vset *ValidatorSet) Proposer() (proposer *Validator) {
return vset.proposer if vset.proposer == nil {
for _, val := range vset.validators {
vset.proposer = vset.proposer.CompareAccum(val)
}
}
return vset.proposer.Copy()
} }
func (vset *ValidatorSet) Hash() []byte { func (vset *ValidatorSet) Hash() []byte {
return vset.validators.Hash() if len(vset.validators) == 0 {
return nil
}
hashables := make([]merkle.Hashable, len(vset.validators))
for i, val := range vset.validators {
hashables[i] = val
}
return merkle.HashFromHashables(hashables)
} }
func (vset *ValidatorSet) Add(val *Validator) (added bool) { func (vset *ValidatorSet) Add(val *Validator) (added bool) {
if vset.validators.Has(val.Id) { val = val.Copy()
idx := sort.Search(len(vset.validators), func(i int) bool {
return val.Id <= vset.validators[i].Id
})
if idx == len(vset.validators) {
vset.validators = append(vset.validators, val)
// Invalidate cache
vset.proposer = nil
vset.totalVotingPower = 0
return true
} else if vset.validators[idx].Id == val.Id {
return false return false
} else {
newValidators := append(vset.validators[:idx], val)
newValidators = append(newValidators, vset.validators[idx:]...)
vset.validators = newValidators
// Invalidate cache
vset.proposer = nil
vset.totalVotingPower = 0
return true
} }
return !vset.validators.Set(val.Id, val)
} }
func (vset *ValidatorSet) Update(val *Validator) (updated bool) { func (vset *ValidatorSet) Update(val *Validator) (updated bool) {
if !vset.validators.Has(val.Id) { index, sameVal := vset.GetById(val.Id)
if sameVal == nil {
return false return false
} else {
vset.validators[index] = val.Copy()
// Invalidate cache
vset.proposer = nil
vset.totalVotingPower = 0
return true
} }
return vset.validators.Set(val.Id, val)
} }
func (vset *ValidatorSet) Remove(validatorId uint64) (val *Validator, removed bool) { func (vset *ValidatorSet) Remove(id uint64) (val *Validator, removed bool) {
val_, removed := vset.validators.Remove(validatorId) idx := sort.Search(len(vset.validators), func(i int) bool {
return val_.(*Validator), removed return id <= vset.validators[i].Id
})
if idx == len(vset.validators) || vset.validators[idx].Id != id {
return nil, false
} else {
removedVal := vset.validators[idx]
newValidators := vset.validators[:idx]
if idx+1 < len(vset.validators) {
newValidators = append(newValidators, vset.validators[idx+1:]...)
}
vset.validators = newValidators
// Invalidate cache
vset.proposer = nil
vset.totalVotingPower = 0
return removedVal, true
}
} }
func (vset *ValidatorSet) Iterate(fn func(index uint, val *Validator) bool) { func (vset *ValidatorSet) Iterate(fn func(index uint, val *Validator) bool) {
index := uint(0) for i, val := range vset.validators {
vset.validators.Iterate(func(key_ interface{}, val_ interface{}) bool { stop := fn(uint(i), val.Copy())
stop := fn(index, val_.(*Validator)) if stop {
index++ break
return stop }
}) }
} }
func (vset *ValidatorSet) String() string { func (vset *ValidatorSet) String() string {
@ -151,7 +239,7 @@ func (vset *ValidatorSet) StringWithIndent(indent string) string {
%s Validators: %s Validators:
%s %v %s %v
%s}`, %s}`,
indent, vset.proposer.String(), indent, vset.Proposer().String(),
indent, indent,
indent, strings.Join(valStrings, "\n"+indent+" "), indent, strings.Join(valStrings, "\n"+indent+" "),
indent) indent)