Remove non-documentation lint

Because of extensive nature of changes, tested all three non-memstore
backends - passed.
This commit is contained in:
kortschak 2014-08-28 11:51:39 +09:30
parent 6614466d23
commit 484bf145a8
35 changed files with 277 additions and 284 deletions

View file

@ -103,7 +103,7 @@ func (it *AllIterator) Next() bool {
} else {
k, _ := cur.Seek(last)
if !bytes.Equal(k, last) {
return fmt.Errorf("Couldn't pick up after", k)
return fmt.Errorf("could not pick up after", k)
}
}
for i < bufferSize {

View file

@ -32,7 +32,7 @@ import (
var (
boltType graph.Type
bufferSize = 50
errNotExist = errors.New("Quad does not exist")
errNotExist = errors.New("quad does not exist")
)
func init() {
@ -43,7 +43,7 @@ type Iterator struct {
uid uint64
tags graph.Tagger
bucket []byte
checkId []byte
checkID []byte
dir quad.Direction
qs *QuadStore
result *Token
@ -56,7 +56,7 @@ type Iterator struct {
func NewIterator(bucket []byte, d quad.Direction, value graph.Value, qs *QuadStore) *Iterator {
tok := value.(*Token)
if !bytes.Equal(tok.bucket, nodeBucket) {
glog.Error("Creating an iterator from a non-node value.")
glog.Error("creating an iterator from a non-node value")
return &Iterator{done: true}
}
@ -68,8 +68,8 @@ func NewIterator(bucket []byte, d quad.Direction, value graph.Value, qs *QuadSto
size: qs.SizeOf(value),
}
it.checkId = make([]byte, len(tok.key))
copy(it.checkId, tok.key)
it.checkID = make([]byte, len(tok.key))
copy(it.checkID, tok.key)
return &it
}
@ -101,7 +101,7 @@ func (it *Iterator) TagResults(dst map[string]graph.Value) {
}
func (it *Iterator) Clone() graph.Iterator {
out := NewIterator(it.bucket, it.dir, &Token{nodeBucket, it.checkId}, it.qs)
out := NewIterator(it.bucket, it.dir, &Token{nodeBucket, it.checkID}, it.qs)
out.Tagger().CopyFrom(it)
return out
}
@ -134,8 +134,8 @@ func (it *Iterator) Next() bool {
b := tx.Bucket(it.bucket)
cur := b.Cursor()
if last == nil {
k, _ := cur.Seek(it.checkId)
if bytes.HasPrefix(k, it.checkId) {
k, _ := cur.Seek(it.checkID)
if bytes.HasPrefix(k, it.checkID) {
var out []byte
out = make([]byte, len(k))
copy(out, k)
@ -148,12 +148,12 @@ func (it *Iterator) Next() bool {
} else {
k, _ := cur.Seek(last)
if !bytes.Equal(k, last) {
return fmt.Errorf("Couldn't pick up after", k)
return fmt.Errorf("could not pick up after", k)
}
}
for i < bufferSize {
k, v := cur.Next()
if k == nil || !bytes.HasPrefix(k, it.checkId) {
if k == nil || !bytes.HasPrefix(k, it.checkID) {
it.buffer = append(it.buffer, nil)
break
}
@ -170,7 +170,7 @@ func (it *Iterator) Next() bool {
})
if err != nil {
if err != errNotExist {
glog.Error("Error nexting in database: ", err)
glog.Errorf("Error nexting in database: %v", err)
}
it.done = true
return false
@ -272,7 +272,7 @@ func (it *Iterator) Contains(v graph.Value) bool {
return false
}
offset := PositionOf(val, it.dir, it.qs)
if bytes.HasPrefix(val.key[offset:], it.checkId) {
if bytes.HasPrefix(val.key[offset:], it.checkID) {
// You may ask, why don't we check to see if it's a valid (not deleted) quad
// again?
//
@ -299,7 +299,7 @@ func (it *Iterator) DebugString(indent int) string {
it.tags.Tags(),
it.dir,
it.size,
it.qs.NameOf(&Token{it.bucket, it.checkId}),
it.qs.NameOf(&Token{it.bucket, it.checkID}),
)
}

View file

@ -101,20 +101,20 @@ func (qs *QuadStore) createBuckets() error {
for _, index := range [][4]quad.Direction{spo, osp, pos, cps} {
_, err = tx.CreateBucket(bucketFor(index))
if err != nil {
return fmt.Errorf("Couldn't create bucket: %s", err)
return fmt.Errorf("could not create bucket: %s", err)
}
}
_, err = tx.CreateBucket(logBucket)
if err != nil {
return fmt.Errorf("Couldn't create bucket: %s", err)
return fmt.Errorf("could not create bucket: %s", err)
}
_, err = tx.CreateBucket(nodeBucket)
if err != nil {
return fmt.Errorf("Couldn't create bucket: %s", err)
return fmt.Errorf("could not create bucket: %s", err)
}
_, err = tx.CreateBucket(metaBucket)
if err != nil {
return fmt.Errorf("Couldn't create bucket: %s", err)
return fmt.Errorf("could not create bucket: %s", err)
}
return nil
})
@ -183,13 +183,13 @@ var (
)
func (qs *QuadStore) ApplyDeltas(deltas []graph.Delta) error {
old_size := qs.size
old_horizon := qs.horizon
oldSize := qs.size
oldHorizon := qs.horizon
err := qs.db.Update(func(tx *bolt.Tx) error {
b := tx.Bucket(logBucket)
b.FillPercent = localFillPercent
resizeMap := make(map[string]int64)
size_change := int64(0)
sizeChange := int64(0)
for _, d := range deltas {
bytes, err := json.Marshal(d)
if err != nil {
@ -215,7 +215,7 @@ func (qs *QuadStore) ApplyDeltas(deltas []graph.Delta) error {
if d.Quad.Label != "" {
resizeMap[d.Quad.Label] += delta
}
size_change += delta
sizeChange += delta
qs.horizon = d.ID
}
for k, v := range resizeMap {
@ -226,14 +226,14 @@ func (qs *QuadStore) ApplyDeltas(deltas []graph.Delta) error {
}
}
}
qs.size += size_change
qs.size += sizeChange
return qs.WriteHorizonAndSize(tx)
})
if err != nil {
glog.Error("Couldn't write to DB for Delta set. Error: ", err)
qs.horizon = old_horizon
qs.size = old_size
qs.horizon = oldHorizon
qs.size = oldSize
return err
}
return nil

View file

@ -258,7 +258,7 @@ func (t Type) String() string {
type StatsContainer struct {
IteratorStats
Kind string
Uid uint64
UID uint64
SubIts []StatsContainer
}
@ -266,7 +266,7 @@ func DumpStats(it Iterator) StatsContainer {
var out StatsContainer
out.IteratorStats = it.Stats()
out.Kind = it.Type().String()
out.Uid = it.UID()
out.UID = it.UID()
for _, sub := range it.SubIterators() {
out.SubIts = append(out.SubIts, DumpStats(sub))
}

View file

@ -110,7 +110,7 @@ func closeIteratorList(its []graph.Iterator, except graph.Iterator) {
// Find if there is a single subiterator which is a valid replacement for this
// And.
func (_ *And) optimizeReplacement(its []graph.Iterator) graph.Iterator {
func (*And) optimizeReplacement(its []graph.Iterator) graph.Iterator {
// If we were created with no SubIterators, we're as good as Null.
if len(its) == 0 {
return &Null{}

View file

@ -20,7 +20,7 @@ import (
)
type Node struct {
Id int `json:"id"`
ID int `json:"id"`
Tags []string `json:"tags,omitempty"`
Values []string `json:"values,omitempty"`
IsLinkNode bool `json:"is_link_node"`
@ -38,15 +38,15 @@ type queryShape struct {
nodes []Node
links []Link
qs graph.QuadStore
nodeId int
hasaIds []int
nodeID int
hasaIDs []int
hasaDirs []quad.Direction
}
func OutputQueryShapeForIterator(it graph.Iterator, qs graph.QuadStore, outputMap map[string]interface{}) {
s := &queryShape{
qs: qs,
nodeId: 1,
nodeID: 1,
}
node := s.MakeNode(it.Clone())
@ -64,16 +64,16 @@ func (s *queryShape) AddLink(l *Link) {
}
func (s *queryShape) LastHasa() (int, quad.Direction) {
return s.hasaIds[len(s.hasaIds)-1], s.hasaDirs[len(s.hasaDirs)-1]
return s.hasaIDs[len(s.hasaIDs)-1], s.hasaDirs[len(s.hasaDirs)-1]
}
func (s *queryShape) PushHasa(i int, d quad.Direction) {
s.hasaIds = append(s.hasaIds, i)
s.hasaIDs = append(s.hasaIDs, i)
s.hasaDirs = append(s.hasaDirs, d)
}
func (s *queryShape) RemoveHasa() {
s.hasaIds = s.hasaIds[:len(s.hasaIds)-1]
s.hasaIDs = s.hasaIDs[:len(s.hasaIDs)-1]
s.hasaDirs = s.hasaDirs[:len(s.hasaDirs)-1]
}
@ -88,16 +88,16 @@ func (s *queryShape) StealNode(left *Node, right *Node) {
left.IsFixed = left.IsFixed || right.IsFixed
for i, link := range s.links {
rewrite := false
if link.LinkNode == right.Id {
link.LinkNode = left.Id
if link.LinkNode == right.ID {
link.LinkNode = left.ID
rewrite = true
}
if link.Source == right.Id {
link.Source = left.Id
if link.Source == right.ID {
link.Source = left.ID
rewrite = true
}
if link.Target == right.Id {
link.Target = left.Id
if link.Target == right.ID {
link.Target = left.ID
rewrite = true
}
if rewrite {
@ -107,24 +107,24 @@ func (s *queryShape) StealNode(left *Node, right *Node) {
}
func (s *queryShape) MakeNode(it graph.Iterator) *Node {
n := Node{Id: s.nodeId}
n := Node{ID: s.nodeID}
for _, tag := range it.Tagger().Tags() {
n.Tags = append(n.Tags, tag)
}
for k, _ := range it.Tagger().Fixed() {
for k := range it.Tagger().Fixed() {
n.Tags = append(n.Tags, k)
}
switch it.Type() {
case graph.And:
for _, sub := range it.SubIterators() {
s.nodeId++
s.nodeID++
newNode := s.MakeNode(sub)
if sub.Type() != graph.Or {
s.StealNode(&n, newNode)
} else {
s.AddNode(newNode)
s.AddLink(&Link{n.Id, newNode.Id, 0, 0})
s.AddLink(&Link{n.ID, newNode.ID, 0, 0})
}
}
case graph.Fixed:
@ -134,35 +134,35 @@ func (s *queryShape) MakeNode(it graph.Iterator) *Node {
}
case graph.HasA:
hasa := it.(*HasA)
s.PushHasa(n.Id, hasa.dir)
s.nodeId++
s.PushHasa(n.ID, hasa.dir)
s.nodeID++
newNode := s.MakeNode(hasa.primaryIt)
s.AddNode(newNode)
s.RemoveHasa()
case graph.Or:
for _, sub := range it.SubIterators() {
s.nodeId++
s.nodeID++
newNode := s.MakeNode(sub)
if sub.Type() == graph.Or {
s.StealNode(&n, newNode)
} else {
s.AddNode(newNode)
s.AddLink(&Link{n.Id, newNode.Id, 0, 0})
s.AddLink(&Link{n.ID, newNode.ID, 0, 0})
}
}
case graph.LinksTo:
n.IsLinkNode = true
lto := it.(*LinksTo)
s.nodeId++
s.nodeID++
newNode := s.MakeNode(lto.primaryIt)
hasaID, hasaDir := s.LastHasa()
if (hasaDir == quad.Subject && lto.dir == quad.Object) ||
(hasaDir == quad.Object && lto.dir == quad.Subject) {
s.AddNode(newNode)
if hasaDir == quad.Subject {
s.AddLink(&Link{hasaID, newNode.Id, 0, n.Id})
s.AddLink(&Link{hasaID, newNode.ID, 0, n.ID})
} else {
s.AddLink(&Link{newNode.Id, hasaID, 0, n.Id})
s.AddLink(&Link{newNode.ID, hasaID, 0, n.ID})
}
} else if lto.primaryIt.Type() == graph.Fixed {
s.StealNode(&n, newNode)

View file

@ -77,14 +77,14 @@ func TestQueryShape(t *testing.T) {
// Link should be correctly typed.
nodes = shape["nodes"].([]Node)
link := shape["links"].([]Link)[0]
if link.Source != nodes[2].Id {
t.Errorf("Failed to get correct link source, got:%v expect:%v", link.Source, nodes[2].Id)
if link.Source != nodes[2].ID {
t.Errorf("Failed to get correct link source, got:%v expect:%v", link.Source, nodes[2].ID)
}
if link.Target != nodes[0].Id {
t.Errorf("Failed to get correct link target, got:%v expect:%v", link.Target, nodes[0].Id)
if link.Target != nodes[0].ID {
t.Errorf("Failed to get correct link target, got:%v expect:%v", link.Target, nodes[0].ID)
}
if link.LinkNode != nodes[1].Id {
t.Errorf("Failed to get correct link node, got:%v expect:%v", link.LinkNode, nodes[1].Id)
if link.LinkNode != nodes[1].ID {
t.Errorf("Failed to get correct link node, got:%v expect:%v", link.LinkNode, nodes[1].ID)
}
if link.Pred != 0 {
t.Errorf("Failed to get correct number of predecessors:%v expect:0", link.Pred)

View file

@ -38,10 +38,10 @@ import (
type Operator int
const (
kCompareLT Operator = iota
kCompareLTE
kCompareGT
kCompareGTE
compareLT Operator = iota
compareLTE
compareGT
compareGTE
// Why no Equals? Because that's usually an AndIterator.
)
@ -99,13 +99,13 @@ func (it *Comparison) Close() {
func RunIntOp(a int64, op Operator, b int64) bool {
switch op {
case kCompareLT:
case compareLT:
return a < b
case kCompareLTE:
case compareLTE:
return a <= b
case kCompareGT:
case compareGT:
return a > b
case kCompareGTE:
case compareGTE:
return a >= b
default:
log.Fatal("Unknown operator type")

View file

@ -40,25 +40,25 @@ var comparisonTests = []struct {
{
message: "successful int64 less than comparison",
operand: int64(3),
operator: kCompareLT,
operator: compareLT,
expect: []string{"0", "1", "2"},
},
{
message: "empty int64 less than comparison",
operand: int64(0),
operator: kCompareLT,
operator: compareLT,
expect: nil,
},
{
message: "successful int64 greater than comparison",
operand: int64(2),
operator: kCompareGT,
operator: compareGT,
expect: []string{"3", "4"},
},
{
message: "successful int64 greater than or equal comparison",
operand: int64(2),
operator: kCompareGTE,
operator: compareGTE,
expect: []string{"2", "3", "4"},
},
}
@ -86,25 +86,25 @@ var vciContainsTests = []struct {
}{
{
message: "1 is less than 2",
operator: kCompareGTE,
operator: compareGTE,
check: 1,
expect: false,
},
{
message: "2 is greater than or equal to 2",
operator: kCompareGTE,
operator: compareGTE,
check: 2,
expect: true,
},
{
message: "3 is greater than or equal to 2",
operator: kCompareGTE,
operator: compareGTE,
check: 3,
expect: true,
},
{
message: "5 is absent from iterator",
operator: kCompareGTE,
operator: compareGTE,
check: 5,
expect: false,
},

View file

@ -33,7 +33,7 @@ type Iterator struct {
uid uint64
tags graph.Tagger
nextPrefix []byte
checkId []byte
checkID []byte
dir quad.Direction
open bool
iter ldbit.Iterator
@ -56,7 +56,7 @@ func NewIterator(prefix string, d quad.Direction, value graph.Value, qs *QuadSto
it := Iterator{
uid: iterator.NextUID(),
nextPrefix: p,
checkId: vb,
checkID: vb,
dir: d,
originalPrefix: prefix,
ro: opts,
@ -106,7 +106,7 @@ func (it *Iterator) TagResults(dst map[string]graph.Value) {
}
func (it *Iterator) Clone() graph.Iterator {
out := NewIterator(it.originalPrefix, it.dir, Token(it.checkId), it.qs)
out := NewIterator(it.originalPrefix, it.dir, Token(it.checkID), it.qs)
out.tags.CopyFrom(it)
return out
}
@ -231,7 +231,7 @@ func (it *Iterator) Contains(v graph.Value) bool {
return false
}
offset := PositionOf(val[0:2], it.dir, it.qs)
if bytes.HasPrefix(val[offset:], it.checkId[1:]) {
if bytes.HasPrefix(val[offset:], it.checkID[1:]) {
// You may ask, why don't we check to see if it's a valid (not deleted) quad
// again?
//
@ -247,7 +247,7 @@ func (it *Iterator) Contains(v graph.Value) bool {
}
func (it *Iterator) Size() (int64, bool) {
return it.qs.SizeOf(Token(it.checkId)), true
return it.qs.SizeOf(Token(it.checkID)), true
}
func (it *Iterator) DebugString(indent int) string {
@ -259,7 +259,7 @@ func (it *Iterator) DebugString(indent int) string {
it.tags.Tags(),
it.dir,
size,
it.qs.NameOf(Token(it.checkId)),
it.qs.NameOf(Token(it.checkID)),
)
}

View file

@ -72,7 +72,7 @@ func createNewLevelDB(path string, _ graph.Options) error {
opts := &opt.Options{}
db, err := leveldb.OpenFile(path, opts)
if err != nil {
glog.Errorf("Error: couldn't create database: %v", err)
glog.Errorf("Error: could not create database: %v", err)
return err
}
defer db.Close()
@ -89,27 +89,27 @@ func newQuadStore(path string, options graph.Options) (graph.QuadStore, error) {
var qs QuadStore
var err error
qs.path = path
cache_size := DefaultCacheSize
cacheSize := DefaultCacheSize
if val, ok := options.IntKey("cache_size_mb"); ok {
cache_size = val
cacheSize = val
}
qs.dbOpts = &opt.Options{
BlockCache: cache.NewLRUCache(cache_size * opt.MiB),
BlockCache: cache.NewLRUCache(cacheSize * opt.MiB),
}
qs.dbOpts.ErrorIfMissing = true
write_buffer_mb := DefaultWriteBufferSize
if val, ok := options.IntKey("write_buffer_mb"); ok {
write_buffer_mb = val
writeBufferSize := DefaultWriteBufferSize
if val, ok := options.IntKey("writeBufferSize"); ok {
writeBufferSize = val
}
qs.dbOpts.WriteBuffer = write_buffer_mb * opt.MiB
qs.dbOpts.WriteBuffer = writeBufferSize * opt.MiB
qs.writeopts = &opt.WriteOptions{
Sync: false,
}
qs.readopts = &opt.ReadOptions{}
db, err := leveldb.OpenFile(qs.path, qs.dbOpts)
if err != nil {
glog.Errorln("Error, couldn't open! ", err)
glog.Errorln("Error, could not open! ", err)
return nil, err
}
qs.db = db
@ -139,13 +139,6 @@ func (qs *QuadStore) Horizon() int64 {
return qs.horizon
}
func (qa *QuadStore) createDeltaKeyFor(d graph.Delta) []byte {
key := make([]byte, 0, 19)
key = append(key, 'd')
key = append(key, []byte(fmt.Sprintf("%018x", d.ID))...)
return key
}
func hashOf(s string) []byte {
h := hashPool.Get().(hash.Hash)
h.Reset()
@ -190,13 +183,13 @@ var (
func (qs *QuadStore) ApplyDeltas(deltas []graph.Delta) error {
batch := &leveldb.Batch{}
resizeMap := make(map[string]int64)
size_change := int64(0)
sizeChange := int64(0)
for _, d := range deltas {
bytes, err := json.Marshal(d)
if err != nil {
return err
}
batch.Put(qs.createDeltaKeyFor(d), bytes)
batch.Put(keyFor(d), bytes)
err = qs.buildQuadWrite(batch, d.Quad, d.ID, d.Action == graph.Add)
if err != nil {
return err
@ -211,7 +204,7 @@ func (qs *QuadStore) ApplyDeltas(deltas []graph.Delta) error {
if d.Quad.Label != "" {
resizeMap[d.Quad.Label] += delta
}
size_change += delta
sizeChange += delta
qs.horizon = d.ID
}
for k, v := range resizeMap {
@ -224,18 +217,25 @@ func (qs *QuadStore) ApplyDeltas(deltas []graph.Delta) error {
}
err := qs.db.Write(batch, qs.writeopts)
if err != nil {
glog.Error("Couldn't write to DB for quadset.")
glog.Error("could not write to DB for quadset.")
return err
}
qs.size += size_change
qs.size += sizeChange
return nil
}
func keyFor(d graph.Delta) []byte {
key := make([]byte, 0, 19)
key = append(key, 'd')
key = append(key, []byte(fmt.Sprintf("%018x", d.ID))...)
return key
}
func (qs *QuadStore) buildQuadWrite(batch *leveldb.Batch, q quad.Quad, id int64, isAdd bool) error {
var entry IndexEntry
data, err := qs.db.Get(qs.createKeyFor(spo, q), qs.readopts)
if err != nil && err != leveldb.ErrNotFound {
glog.Error("Couldn't access DB to prepare index: ", err)
glog.Error("could not access DB to prepare index: ", err)
return err
}
if err == nil {
@ -251,12 +251,12 @@ func (qs *QuadStore) buildQuadWrite(batch *leveldb.Batch, q quad.Quad, id int64,
if isAdd && len(entry.History)%2 == 0 {
glog.Error("Entry History is out of sync for", entry)
return errors.New("Odd index history")
return errors.New("odd index history")
}
bytes, err := json.Marshal(entry)
if err != nil {
glog.Errorf("Couldn't write to buffer for entry %#v: %s", entry, err)
glog.Errorf("could not write to buffer for entry %#v: %s", entry, err)
return err
}
batch.Put(qs.createKeyFor(spo, q), bytes)
@ -288,7 +288,7 @@ func (qs *QuadStore) UpdateValueKeyBy(name string, amount int64, batch *leveldb.
if b != nil && err != leveldb.ErrNotFound {
err = json.Unmarshal(b, value)
if err != nil {
glog.Errorf("Error: couldn't reconstruct value: %v", err)
glog.Errorf("Error: could not reconstruct value: %v", err)
return err
}
value.Size += amount
@ -302,7 +302,7 @@ func (qs *QuadStore) UpdateValueKeyBy(name string, amount int64, batch *leveldb.
// Repackage and rewrite.
bytes, err := json.Marshal(&value)
if err != nil {
glog.Errorf("Couldn't write to buffer for value %s: %s", name, err)
glog.Errorf("could not write to buffer for value %s: %s", name, err)
return err
}
if batch == nil {
@ -319,20 +319,20 @@ func (qs *QuadStore) Close() {
if err == nil {
werr := qs.db.Put([]byte("__size"), buf.Bytes(), qs.writeopts)
if werr != nil {
glog.Error("Couldn't write size before closing!")
glog.Error("could not write size before closing!")
}
} else {
glog.Errorf("Couldn't convert size before closing!")
glog.Errorf("could not convert size before closing!")
}
buf.Reset()
err = binary.Write(buf, binary.LittleEndian, qs.horizon)
if err == nil {
werr := qs.db.Put([]byte("__horizon"), buf.Bytes(), qs.writeopts)
if werr != nil {
glog.Error("Couldn't write horizon before closing!")
glog.Error("could not write horizon before closing!")
}
} else {
glog.Errorf("Couldn't convert horizon before closing!")
glog.Errorf("could not convert horizon before closing!")
}
qs.db.Close()
qs.open = false
@ -342,7 +342,7 @@ func (qs *QuadStore) Quad(k graph.Value) quad.Quad {
var q quad.Quad
b, err := qs.db.Get(k.(Token), qs.readopts)
if err != nil && err != leveldb.ErrNotFound {
glog.Error("Error: couldn't get quad from DB.")
glog.Error("Error: could not get quad from DB.")
return quad.Quad{}
}
if err == leveldb.ErrNotFound {
@ -351,7 +351,7 @@ func (qs *QuadStore) Quad(k graph.Value) quad.Quad {
}
err = json.Unmarshal(b, &q)
if err != nil {
glog.Error("Error: couldn't reconstruct quad.")
glog.Error("Error: could not reconstruct quad.")
return quad.Quad{}
}
return q
@ -361,20 +361,20 @@ func (qs *QuadStore) ValueOf(s string) graph.Value {
return Token(qs.createValueKeyFor(s))
}
func (qs *QuadStore) valueData(value_key []byte) ValueData {
func (qs *QuadStore) valueData(key []byte) ValueData {
var out ValueData
if glog.V(3) {
glog.V(3).Infof("%s %v", string(value_key[0]), value_key)
glog.V(3).Infof("%c %v", key[0], key)
}
b, err := qs.db.Get(value_key, qs.readopts)
b, err := qs.db.Get(key, qs.readopts)
if err != nil && err != leveldb.ErrNotFound {
glog.Errorln("Error: couldn't get value from DB")
glog.Errorln("Error: could not get value from DB")
return out
}
if b != nil && err != leveldb.ErrNotFound {
err = json.Unmarshal(b, &out)
if err != nil {
glog.Errorln("Error: couldn't reconstruct value")
glog.Errorln("Error: could not reconstruct value")
return ValueData{}
}
}
@ -400,7 +400,7 @@ func (qs *QuadStore) getInt64ForKey(key string, empty int64) (int64, error) {
var out int64
b, err := qs.db.Get([]byte(key), qs.readopts)
if err != nil && err != leveldb.ErrNotFound {
glog.Errorln("Couldn't read " + key + ": " + err.Error())
glog.Errorln("could not read " + key + ": " + err.Error())
return 0, err
}
if err == leveldb.ErrNotFound {
@ -410,7 +410,7 @@ func (qs *QuadStore) getInt64ForKey(key string, empty int64) (int64, error) {
buf := bytes.NewBuffer(b)
err = binary.Read(buf, binary.LittleEndian, &out)
if err != nil {
glog.Errorln("Error: couldn't parse", key)
glog.Errorln("Error: could not parse", key)
return 0, err
}
return out, nil
@ -471,9 +471,8 @@ func (qs *QuadStore) QuadDirection(val graph.Value, d quad.Direction) graph.Valu
offset := PositionOf(v[0:2], d, qs)
if offset != -1 {
return Token(append([]byte("z"), v[offset:offset+hashSize]...))
} else {
return Token(qs.Quad(val).Get(d))
}
return Token(qs.Quad(val).Get(d))
}
func compareBytes(a, b graph.Value) bool {

View file

@ -31,7 +31,7 @@ type (
func newNodesAllIterator(qs *QuadStore) *nodesAllIterator {
var out nodesAllIterator
out.Int64 = *iterator.NewInt64(1, qs.idCounter-1)
out.Int64 = *iterator.NewInt64(1, qs.nextID-1)
out.qs = qs
return &out
}
@ -45,7 +45,7 @@ func (it *nodesAllIterator) Next() bool {
if !it.Int64.Next() {
return false
}
_, ok := it.qs.revIdMap[it.Int64.Result().(int64)]
_, ok := it.qs.revIDMap[it.Int64.Result().(int64)]
if !ok {
return it.Next()
}
@ -54,7 +54,7 @@ func (it *nodesAllIterator) Next() bool {
func newQuadsAllIterator(qs *QuadStore) *quadsAllIterator {
var out quadsAllIterator
out.Int64 = *iterator.NewInt64(1, qs.quadIdCounter-1)
out.Int64 = *iterator.NewInt64(1, qs.nextQuadID-1)
out.qs = qs
return &out
}

View file

@ -70,27 +70,27 @@ type LogEntry struct {
}
type QuadStore struct {
idCounter int64
quadIdCounter int64
idMap map[string]int64
revIdMap map[int64]string
log []LogEntry
size int64
index QuadDirectionIndex
nextID int64
nextQuadID int64
idMap map[string]int64
revIDMap map[int64]string
log []LogEntry
size int64
index QuadDirectionIndex
// vip_index map[string]map[int64]map[string]map[int64]*b.Tree
}
func newQuadStore() *QuadStore {
return &QuadStore{
idMap: make(map[string]int64),
revIdMap: make(map[int64]string),
revIDMap: make(map[int64]string),
// Sentinel null entry so indices start at 1
log: make([]LogEntry, 1, 200),
index: NewQuadDirectionIndex(),
idCounter: 1,
quadIdCounter: 1,
index: NewQuadDirectionIndex(),
nextID: 1,
nextQuadID: 1,
}
}
@ -148,10 +148,10 @@ func (qs *QuadStore) AddDelta(d graph.Delta) error {
if _, exists := qs.indexOf(d.Quad); exists {
return graph.ErrQuadExists
}
qid := qs.quadIdCounter
qid := qs.nextQuadID
qs.log = append(qs.log, LogEntry{Delta: d})
qs.size++
qs.quadIdCounter++
qs.nextQuadID++
for dir := quad.Subject; dir <= quad.Label; dir++ {
sid := d.Quad.Get(dir)
@ -159,9 +159,9 @@ func (qs *QuadStore) AddDelta(d graph.Delta) error {
continue
}
if _, ok := qs.idMap[sid]; !ok {
qs.idMap[sid] = qs.idCounter
qs.revIdMap[qs.idCounter] = sid
qs.idCounter++
qs.idMap[sid] = qs.nextID
qs.revIDMap[qs.nextID] = sid
qs.nextID++
}
}
@ -184,11 +184,11 @@ func (qs *QuadStore) RemoveDelta(d graph.Delta) error {
return graph.ErrQuadNotExist
}
quadID := qs.quadIdCounter
quadID := qs.nextQuadID
qs.log = append(qs.log, LogEntry{Delta: d})
qs.log[prevQuadID].DeletedBy = quadID
qs.size--
qs.quadIdCounter++
qs.nextQuadID++
return nil
}
@ -227,7 +227,7 @@ func (qs *QuadStore) ValueOf(name string) graph.Value {
}
func (qs *QuadStore) NameOf(id graph.Value) string {
return qs.revIdMap[id.(int64)]
return qs.revIDMap[id.(int64)]
}
func (qs *QuadStore) QuadsAllIterator() graph.Iterator {

View file

@ -164,11 +164,11 @@ func TestLinksToOptimization(t *testing.T) {
}
v := newIt.(*Iterator)
v_clone := v.Clone()
if v_clone.DebugString(0) != v.DebugString(0) {
t.Fatal("Wrong iterator. Got ", v_clone.DebugString(0))
vClone := v.Clone()
if vClone.DebugString(0) != v.DebugString(0) {
t.Fatal("Wrong iterator. Got ", vClone.DebugString(0))
}
vt := v_clone.Tagger()
vt := vClone.Tagger()
if len(vt.Tags()) < 1 || vt.Tags()[0] != "foo" {
t.Fatal("Tag on LinksTo did not persist")
}

View file

@ -130,7 +130,7 @@ func (it *Iterator) Clone() graph.Iterator {
func (it *Iterator) Next() bool {
var result struct {
Id string `bson:"_id"`
ID string `bson:"_id"`
Added []int64 `bson:"Added"`
Deleted []int64 `bson:"Deleted"`
}
@ -145,7 +145,7 @@ func (it *Iterator) Next() bool {
if it.collection == "quads" && len(result.Added) <= len(result.Deleted) {
return it.Next()
}
it.result = result.Id
it.result = result.ID
return true
}

View file

@ -18,45 +18,48 @@ import (
"container/list"
)
type IDLru struct {
// TODO(kortschak) Reimplement without container/list.
// cache implements an LRU cache.
type cache struct {
cache map[string]*list.Element
priority *list.List
maxSize int
}
type KV struct {
type kv struct {
key string
value string
}
func NewIDLru(size int) *IDLru {
var lru IDLru
func newCache(size int) *cache {
var lru cache
lru.maxSize = size
lru.priority = list.New()
lru.cache = make(map[string]*list.Element)
return &lru
}
func (lru *IDLru) Put(key string, value string) {
func (lru *cache) Put(key string, value string) {
if _, ok := lru.Get(key); ok {
return
}
if len(lru.cache) == lru.maxSize {
lru.removeOldest()
}
lru.priority.PushFront(KV{key: key, value: value})
lru.priority.PushFront(kv{key: key, value: value})
lru.cache[key] = lru.priority.Front()
}
func (lru *IDLru) Get(key string) (string, bool) {
func (lru *cache) Get(key string) (string, bool) {
if element, ok := lru.cache[key]; ok {
lru.priority.MoveToFront(element)
return element.Value.(KV).value, true
return element.Value.(kv).value, true
}
return "", false
}
func (lru *IDLru) removeOldest() {
func (lru *cache) removeOldest() {
last := lru.priority.Remove(lru.priority.Back())
delete(lru.cache, last.(KV).key)
delete(lru.cache, last.(kv).key)
}

View file

@ -45,7 +45,7 @@ var (
type QuadStore struct {
session *mgo.Session
db *mgo.Database
idCache *IDLru
ids *cache
}
func createNewMongoGraph(addr string, options graph.Options) error {
@ -97,11 +97,11 @@ func newQuadStore(addr string, options graph.Options) (graph.QuadStore, error) {
}
qs.db = conn.DB(dbName)
qs.session = conn
qs.idCache = NewIDLru(1 << 16)
qs.ids = newCache(1 << 16)
return &qs, nil
}
func (qs *QuadStore) getIdForQuad(t quad.Quad) string {
func (qs *QuadStore) getIDForQuad(t quad.Quad) string {
id := hashOf(t.Subject)
id += hashOf(t.Predicate)
id += hashOf(t.Object)
@ -121,7 +121,7 @@ func hashOf(s string) string {
}
type MongoNode struct {
Id string `bson:"_id"`
ID string `bson:"_id"`
Name string `bson:"Name"`
Size int `bson:"Size"`
}
@ -133,11 +133,11 @@ type MongoLogEntry struct {
Timestamp int64
}
func (qs *QuadStore) updateNodeBy(node_name string, inc int) error {
node := qs.ValueOf(node_name)
func (qs *QuadStore) updateNodeBy(name string, inc int) error {
node := qs.ValueOf(name)
doc := bson.M{
"_id": node.(string),
"Name": node_name,
"Name": name,
}
upsert := bson.M{
"$setOnInsert": doc,
@ -166,7 +166,7 @@ func (qs *QuadStore) updateQuad(q quad.Quad, id int64, proc graph.Procedure) err
setname: id,
},
}
_, err := qs.db.C("quads").UpsertId(qs.getIdForQuad(q), upsert)
_, err := qs.db.C("quads").UpsertId(qs.getIDForQuad(q), upsert)
if err != nil {
glog.Errorf("Error: %v", err)
}
@ -202,7 +202,7 @@ func (qs *QuadStore) updateLog(d graph.Delta) error {
entry := MongoLogEntry{
LogID: d.ID,
Action: action,
Key: qs.getIdForQuad(d.Quad),
Key: qs.getIDForQuad(d.Quad),
Timestamp: d.Timestamp.UnixNano(),
}
err := qs.db.C("log").Insert(entry)
@ -217,7 +217,7 @@ func (qs *QuadStore) ApplyDeltas(in []graph.Delta) error {
ids := make(map[string]int)
// Pre-check the existence condition.
for _, d := range in {
key := qs.getIdForQuad(d.Quad)
key := qs.getIDForQuad(d.Quad)
switch d.Action {
case graph.Add:
if qs.checkValid(key) {
@ -292,7 +292,7 @@ func (qs *QuadStore) ValueOf(s string) graph.Value {
}
func (qs *QuadStore) NameOf(v graph.Value) string {
val, ok := qs.idCache.Get(v.(string))
val, ok := qs.ids.Get(v.(string))
if ok {
return val
}
@ -301,7 +301,7 @@ func (qs *QuadStore) NameOf(v graph.Value) string {
if err != nil {
glog.Errorf("Error: Couldn't retrieve node %s %v", v, err)
}
qs.idCache.Put(v.(string), node.Name)
qs.ids.Put(v.(string), node.Name)
return node.Name
}

View file

@ -54,8 +54,8 @@ func (h *Handle) Close() {
}
var (
ErrQuadExists = errors.New("Quad exists")
ErrQuadNotExist = errors.New("Quad doesn't exist")
ErrQuadExists = errors.New("quad exists")
ErrQuadNotExist = errors.New("quad does not exist")
)
type QuadWriter interface {