Merge branch 'master' into nexter

Conflicts:
	graph/leveldb/all_iterator.go
	graph/leveldb/iterator.go
	graph/memstore/triplestore.go
	query/gremlin/finals.go
This commit is contained in:
kortschak 2014-08-07 06:52:36 +09:30
commit 62785d25c2
37 changed files with 882 additions and 467 deletions

View file

@ -157,6 +157,22 @@ func Next(it Iterator) bool {
return false
}
// Height is a convienence function to measure the height of an iterator tree.
func Height(it Iterator, until Type) int {
if it.Type() == until {
return 1
}
subs := it.SubIterators()
maxDepth := 0
for _, sub := range subs {
h := Height(sub, until)
if h > maxDepth {
maxDepth = h
}
}
return maxDepth + 1
}
// FixedIterator wraps iterators that are modifiable by addition of fixed value sets.
type FixedIterator interface {
Iterator
@ -184,6 +200,7 @@ const (
Fixed
Not
Optional
Materialize
)
var (
@ -204,6 +221,7 @@ var (
"fixed",
"not",
"optional",
"materialize",
}
)

View file

@ -70,6 +70,8 @@ func (it *And) Optimize() (graph.Iterator, bool) {
// now a permutation of itself, but the contents are unchanged.
its = optimizeOrder(its)
its = materializeIts(its)
// Okay! At this point we have an optimized order.
// The easiest thing to do at this point is merely to create a new And iterator
@ -293,6 +295,21 @@ func hasOneUsefulIterator(its []graph.Iterator) graph.Iterator {
return nil
}
func materializeIts(its []graph.Iterator) []graph.Iterator {
var out []graph.Iterator
for _, it := range its {
stats := it.Stats()
if stats.Size*stats.NextCost < stats.ContainsCost {
if graph.Height(it, graph.Materialize) > 10 {
out = append(out, NewMaterialize(it))
continue
}
}
out = append(out, it)
}
return out
}
// and.Stats() lives here in and-iterator-optimize.go because it may
// in the future return different statistics based on how it is optimized.
// For now, however, it's pretty static.

View file

@ -0,0 +1,273 @@
// Copyright 2014 The Cayley Authors. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package iterator
// A simple iterator that, when first called Contains() or Next() upon, materializes the whole subiterator, stores it locally, and responds. Essentially a cache.
import (
"fmt"
"strings"
"github.com/barakmich/glog"
"github.com/google/cayley/graph"
)
var abortMaterializeAt = 1000
type result struct {
id graph.Value
tags map[string]graph.Value
}
type Keyer interface {
Key() interface{}
}
type Materialize struct {
uid uint64
tags graph.Tagger
containsMap map[graph.Value]int
values [][]result
index int
subindex int
subIt graph.Iterator
hasRun bool
aborted bool
}
func NewMaterialize(sub graph.Iterator) *Materialize {
return &Materialize{
uid: NextUID(),
containsMap: make(map[graph.Value]int),
subIt: sub,
index: -1,
}
}
func (it *Materialize) UID() uint64 {
return it.uid
}
func (it *Materialize) Reset() {
it.subIt.Reset()
it.index = -1
}
func (it *Materialize) Close() {
it.subIt.Close()
it.containsMap = nil
it.values = nil
it.hasRun = false
}
func (it *Materialize) Tagger() *graph.Tagger {
return &it.tags
}
func (it *Materialize) TagResults(dst map[string]graph.Value) {
if !it.hasRun {
return
}
if it.aborted {
it.subIt.TagResults(dst)
return
}
if it.Result() == nil {
return
}
for _, tag := range it.tags.Tags() {
dst[tag] = it.Result()
}
for tag, value := range it.values[it.index][it.subindex].tags {
dst[tag] = value
}
}
func (it *Materialize) Clone() graph.Iterator {
out := NewMaterialize(it.subIt.Clone())
out.tags.CopyFrom(it)
if it.hasRun {
out.hasRun = true
out.aborted = it.aborted
out.values = it.values
out.containsMap = it.containsMap
}
return out
}
// Print some information about the iterator.
func (it *Materialize) DebugString(indent int) string {
return fmt.Sprintf("%s(%s tags: %s Size: %d\n%s)",
strings.Repeat(" ", indent),
it.Type(),
it.tags.Tags(),
len(it.values),
it.subIt.DebugString(indent+4),
)
}
// Register this iterator as a Materialize iterator.
func (it *Materialize) Type() graph.Type { return graph.Materialize }
// DEPRECATED
func (it *Materialize) ResultTree() *graph.ResultTree {
tree := graph.NewResultTree(it.Result())
tree.AddSubtree(it.subIt.ResultTree())
return tree
}
func (it *Materialize) Result() graph.Value {
if len(it.values) == 0 {
return nil
}
if it.index == -1 {
return nil
}
if it.index >= len(it.values) {
return nil
}
return it.values[it.index][it.subindex].id
}
func (it *Materialize) SubIterators() []graph.Iterator {
return []graph.Iterator{it.subIt}
}
func (it *Materialize) Optimize() (graph.Iterator, bool) {
newSub, changed := it.subIt.Optimize()
if changed {
it.subIt = newSub
if it.subIt.Type() == graph.Null {
return it.subIt, true
}
}
return it, false
}
// Size is the number of values stored, if we've got them all.
// Otherwise, guess based on the size of the subiterator.
func (it *Materialize) Size() (int64, bool) {
if it.hasRun {
return int64(len(it.values)), true
}
return it.subIt.Size()
}
// The entire point of Materialize is to amortize the cost by
// putting it all up front.
func (it *Materialize) Stats() graph.IteratorStats {
overhead := int64(2)
size, _ := it.Size()
subitStats := it.subIt.Stats()
return graph.IteratorStats{
ContainsCost: overhead * subitStats.NextCost,
NextCost: overhead * subitStats.NextCost,
Size: size,
}
}
func (it *Materialize) Next() (graph.Value, bool) {
graph.NextLogIn(it)
if !it.hasRun {
it.materializeSet()
}
if it.aborted {
return graph.Next(it.subIt)
}
it.index++
it.subindex = 0
if it.index >= len(it.values) {
return graph.NextLogOut(it, nil, false)
}
return graph.NextLogOut(it, it.Result(), true)
}
func (it *Materialize) Contains(v graph.Value) bool {
graph.ContainsLogIn(it, v)
if !it.hasRun {
it.materializeSet()
}
if it.aborted {
return it.subIt.Contains(v)
}
key := v
if h, ok := v.(Keyer); ok {
key = h.Key()
}
if i, ok := it.containsMap[key]; ok {
it.index = i
it.subindex = 0
return graph.ContainsLogOut(it, v, true)
}
return graph.ContainsLogOut(it, v, false)
}
func (it *Materialize) NextResult() bool {
if !it.hasRun {
it.materializeSet()
}
if it.aborted {
return it.subIt.NextResult()
}
it.subindex++
if it.subindex >= len(it.values[it.index]) {
// Don't go off the end of the world
it.subindex--
return false
}
return true
}
func (it *Materialize) materializeSet() {
i := 0
for {
id, ok := graph.Next(it.subIt)
if !ok {
break
}
i += 1
if i > abortMaterializeAt {
it.aborted = true
break
}
val := id
if h, ok := id.(Keyer); ok {
val = h.Key()
}
if _, ok := it.containsMap[val]; !ok {
it.containsMap[val] = len(it.values)
it.values = append(it.values, nil)
}
index := it.containsMap[val]
tags := make(map[string]graph.Value)
it.subIt.TagResults(tags)
it.values[index] = append(it.values[index], result{id: id, tags: tags})
for it.subIt.NextResult() == true {
tags := make(map[string]graph.Value)
it.subIt.TagResults(tags)
it.values[index] = append(it.values[index], result{id: id, tags: tags})
}
}
if it.aborted {
it.values = nil
it.containsMap = nil
it.subIt.Reset()
}
glog.Infof("Materialization List %d: %#v", it.values)
it.hasRun = true
}

View file

@ -36,11 +36,11 @@ func (qs *store) ValueOf(s string) graph.Value {
return nil
}
func (qs *store) AddTriple(*quad.Quad) {}
func (qs *store) AddTriple(quad.Quad) {}
func (qs *store) AddTripleSet([]*quad.Quad) {}
func (qs *store) AddTripleSet([]quad.Quad) {}
func (qs *store) Quad(graph.Value) *quad.Quad { return &quad.Quad{} }
func (qs *store) Quad(graph.Value) quad.Quad { return quad.Quad{} }
func (qs *store) TripleIterator(d quad.Direction, i graph.Value) graph.Iterator {
return qs.iter
@ -74,4 +74,4 @@ func (qs *store) Close() {}
func (qs *store) TripleDirection(graph.Value, quad.Direction) graph.Value { return 0 }
func (qs *store) RemoveTriple(t *quad.Quad) {}
func (qs *store) RemoveTriple(t quad.Quad) {}

View file

@ -117,7 +117,7 @@ func (it *AllIterator) Next() bool {
it.Close()
return false
}
it.result = out
it.result = Token(out)
return true
}

View file

@ -42,7 +42,7 @@ type Iterator struct {
}
func NewIterator(prefix string, d quad.Direction, value graph.Value, qs *TripleStore) *Iterator {
vb := value.([]byte)
vb := value.(Token)
p := make([]byte, 0, 2+qs.hasher.Size())
p = append(p, []byte(prefix)...)
p = append(p, []byte(vb[1:])...)
@ -105,7 +105,7 @@ func (it *Iterator) TagResults(dst map[string]graph.Value) {
}
func (it *Iterator) Clone() graph.Iterator {
out := NewIterator(it.originalPrefix, it.dir, it.checkId, it.qs)
out := NewIterator(it.originalPrefix, it.dir, Token(it.checkId), it.qs)
out.tags.CopyFrom(it)
return out
}
@ -134,7 +134,7 @@ func (it *Iterator) Next() bool {
if bytes.HasPrefix(it.iter.Key(), it.nextPrefix) {
out := make([]byte, len(it.iter.Key()))
copy(out, it.iter.Key())
it.result = out
it.result = Token(out)
ok := it.iter.Next()
if !ok {
it.Close()
@ -216,7 +216,7 @@ func PositionOf(prefix []byte, d quad.Direction, qs *TripleStore) int {
}
func (it *Iterator) Contains(v graph.Value) bool {
val := v.([]byte)
val := v.(Token)
if val[0] == 'z' {
return false
}
@ -227,7 +227,7 @@ func (it *Iterator) Contains(v graph.Value) bool {
}
} else {
nameForDir := it.qs.Quad(v).Get(it.dir)
hashForDir := it.qs.ValueOf(nameForDir).([]byte)
hashForDir := it.qs.ValueOf(nameForDir).(Token)
if bytes.Equal(hashForDir, it.checkId) {
return true
}
@ -236,12 +236,20 @@ func (it *Iterator) Contains(v graph.Value) bool {
}
func (it *Iterator) Size() (int64, bool) {
return it.qs.SizeOf(it.checkId), true
return it.qs.SizeOf(Token(it.checkId)), true
}
func (it *Iterator) DebugString(indent int) string {
size, _ := it.Size()
return fmt.Sprintf("%s(%s %d tags: %v dir: %s size:%d %s)", strings.Repeat(" ", indent), it.Type(), it.UID(), it.tags.Tags(), it.dir, size, it.qs.NameOf(it.checkId))
return fmt.Sprintf("%s(%s %d tags: %v dir: %s size:%d %s)",
strings.Repeat(" ", indent),
it.Type(),
it.UID(),
it.tags.Tags(),
it.dir,
size,
it.qs.NameOf(Token(it.checkId)),
)
}
var levelDBType graph.Type

View file

@ -26,8 +26,8 @@ import (
"github.com/google/cayley/quad"
)
func makeTripleSet() []*quad.Quad {
tripleSet := []*quad.Quad{
func makeTripleSet() []quad.Quad {
tripleSet := []quad.Quad{
{"A", "follows", "B", ""},
{"C", "follows", "B", ""},
{"C", "follows", "D", ""},
@ -43,7 +43,7 @@ func makeTripleSet() []*quad.Quad {
return tripleSet
}
func iteratedTriples(qs graph.TripleStore, it graph.Iterator) []*quad.Quad {
func iteratedTriples(qs graph.TripleStore, it graph.Iterator) []quad.Quad {
var res ordered
for graph.Next(it) {
res = append(res, qs.Quad(it.Result()))
@ -52,7 +52,7 @@ func iteratedTriples(qs graph.TripleStore, it graph.Iterator) []*quad.Quad {
return res
}
type ordered []*quad.Quad
type ordered []quad.Quad
func (o ordered) Len() int { return len(o) }
func (o ordered) Less(i, j int) bool {
@ -135,7 +135,7 @@ func TestLoadDatabase(t *testing.T) {
t.Error("Failed to create leveldb TripleStore.")
}
qs.AddTriple(&quad.Quad{"Something", "points_to", "Something Else", "context"})
qs.AddTriple(quad.Quad{"Something", "points_to", "Something Else", "context"})
for _, pq := range []string{"Something", "points_to", "Something Else", "context"} {
if got := qs.NameOf(qs.ValueOf(pq)); got != pq {
t.Errorf("Failed to roundtrip %q, got:%q expect:%q", pq, got, pq)
@ -168,7 +168,7 @@ func TestLoadDatabase(t *testing.T) {
t.Errorf("Unexpected triplestore size, got:%d expect:5", s)
}
qs.RemoveTriple(&quad.Quad{"A", "follows", "B", ""})
qs.RemoveTriple(quad.Quad{"A", "follows", "B", ""})
if s := qs.Size(); s != 10 {
t.Errorf("Unexpected triplestore size after RemoveTriple, got:%d expect:10", s)
}
@ -293,7 +293,7 @@ func TestSetIterator(t *testing.T) {
qs.AddTripleSet(makeTripleSet())
expect := []*quad.Quad{
expect := []quad.Quad{
{"C", "follows", "B", ""},
{"C", "follows", "D", ""},
}
@ -318,7 +318,7 @@ func TestSetIterator(t *testing.T) {
// Object iterator.
it = qs.TripleIterator(quad.Object, qs.ValueOf("F"))
expect = []*quad.Quad{
expect = []quad.Quad{
{"B", "follows", "F", ""},
{"E", "follows", "F", ""},
}
@ -331,7 +331,7 @@ func TestSetIterator(t *testing.T) {
and.AddSubIterator(qs.TripleIterator(quad.Subject, qs.ValueOf("B")))
and.AddSubIterator(it)
expect = []*quad.Quad{
expect = []quad.Quad{
{"B", "follows", "F", ""},
}
if got := iteratedTriples(qs, and); !reflect.DeepEqual(got, expect) {
@ -341,7 +341,7 @@ func TestSetIterator(t *testing.T) {
// Predicate iterator.
it = qs.TripleIterator(quad.Predicate, qs.ValueOf("status"))
expect = []*quad.Quad{
expect = []quad.Quad{
{"B", "status", "cool", "status_graph"},
{"D", "status", "cool", "status_graph"},
{"G", "status", "cool", "status_graph"},
@ -354,7 +354,7 @@ func TestSetIterator(t *testing.T) {
// Label iterator.
it = qs.TripleIterator(quad.Label, qs.ValueOf("status_graph"))
expect = []*quad.Quad{
expect = []quad.Quad{
{"B", "status", "cool", "status_graph"},
{"D", "status", "cool", "status_graph"},
{"G", "status", "cool", "status_graph"},
@ -370,7 +370,7 @@ func TestSetIterator(t *testing.T) {
and.AddSubIterator(qs.TripleIterator(quad.Subject, qs.ValueOf("B")))
and.AddSubIterator(it)
expect = []*quad.Quad{
expect = []quad.Quad{
{"B", "status", "cool", "status_graph"},
}
if got := iteratedTriples(qs, and); !reflect.DeepEqual(got, expect) {
@ -383,7 +383,7 @@ func TestSetIterator(t *testing.T) {
and.AddSubIterator(it)
and.AddSubIterator(qs.TripleIterator(quad.Subject, qs.ValueOf("B")))
expect = []*quad.Quad{
expect = []quad.Quad{
{"B", "status", "cool", "status_graph"},
}
if got := iteratedTriples(qs, and); !reflect.DeepEqual(got, expect) {

View file

@ -42,6 +42,12 @@ const (
DefaultWriteBufferSize = 20
)
type Token []byte
func (t Token) Key() interface{} {
return string(t)
}
type TripleStore struct {
dbOpts *opt.Options
db *leveldb.DB
@ -116,7 +122,7 @@ func (qs *TripleStore) Size() int64 {
return qs.size
}
func (qs *TripleStore) createKeyFor(d [3]quad.Direction, triple *quad.Quad) []byte {
func (qs *TripleStore) createKeyFor(d [3]quad.Direction, triple quad.Quad) []byte {
key := make([]byte, 0, 2+(qs.hasher.Size()*3))
// TODO(kortschak) Remove dependence on String() method.
key = append(key, []byte{d[0].Prefix(), d[1].Prefix()}...)
@ -126,7 +132,7 @@ func (qs *TripleStore) createKeyFor(d [3]quad.Direction, triple *quad.Quad) []by
return key
}
func (qs *TripleStore) createProvKeyFor(d [3]quad.Direction, triple *quad.Quad) []byte {
func (qs *TripleStore) createProvKeyFor(d [3]quad.Direction, triple quad.Quad) []byte {
key := make([]byte, 0, 2+(qs.hasher.Size()*4))
// TODO(kortschak) Remove dependence on String() method.
key = append(key, []byte{quad.Label.Prefix(), d[0].Prefix()}...)
@ -144,7 +150,7 @@ func (qs *TripleStore) createValueKeyFor(s string) []byte {
return key
}
func (qs *TripleStore) AddTriple(t *quad.Quad) {
func (qs *TripleStore) AddTriple(t quad.Quad) {
batch := &leveldb.Batch{}
qs.buildWrite(batch, t)
err := qs.db.Write(batch, qs.writeopts)
@ -163,7 +169,7 @@ var (
pso = [3]quad.Direction{quad.Predicate, quad.Subject, quad.Object}
)
func (qs *TripleStore) RemoveTriple(t *quad.Quad) {
func (qs *TripleStore) RemoveTriple(t quad.Quad) {
_, err := qs.db.Get(qs.createKeyFor(spo, t), qs.readopts)
if err != nil && err != leveldb.ErrNotFound {
glog.Error("Couldn't access DB to confirm deletion")
@ -192,8 +198,8 @@ func (qs *TripleStore) RemoveTriple(t *quad.Quad) {
qs.size--
}
func (qs *TripleStore) buildTripleWrite(batch *leveldb.Batch, t *quad.Quad) {
bytes, err := json.Marshal(*t)
func (qs *TripleStore) buildTripleWrite(batch *leveldb.Batch, t quad.Quad) {
bytes, err := json.Marshal(t)
if err != nil {
glog.Errorf("Couldn't write to buffer for triple %s: %s", t, err)
return
@ -206,7 +212,7 @@ func (qs *TripleStore) buildTripleWrite(batch *leveldb.Batch, t *quad.Quad) {
}
}
func (qs *TripleStore) buildWrite(batch *leveldb.Batch, t *quad.Quad) {
func (qs *TripleStore) buildWrite(batch *leveldb.Batch, t quad.Quad) {
qs.buildTripleWrite(batch, t)
qs.UpdateValueKeyBy(t.Get(quad.Subject), 1, nil)
qs.UpdateValueKeyBy(t.Get(quad.Predicate), 1, nil)
@ -267,7 +273,7 @@ func (qs *TripleStore) UpdateValueKeyBy(name string, amount int, batch *leveldb.
}
}
func (qs *TripleStore) AddTripleSet(t_s []*quad.Quad) {
func (qs *TripleStore) AddTripleSet(t_s []quad.Quad) {
batch := &leveldb.Batch{}
newTs := len(t_s)
resizeMap := make(map[string]int)
@ -306,23 +312,23 @@ func (qs *TripleStore) Close() {
qs.open = false
}
func (qs *TripleStore) Quad(k graph.Value) *quad.Quad {
func (qs *TripleStore) Quad(k graph.Value) quad.Quad {
var triple quad.Quad
b, err := qs.db.Get(k.([]byte), qs.readopts)
b, err := qs.db.Get(k.(Token), qs.readopts)
if err != nil && err != leveldb.ErrNotFound {
glog.Error("Error: couldn't get triple from DB.")
return &quad.Quad{}
return quad.Quad{}
}
if err == leveldb.ErrNotFound {
// No harm, no foul.
return &quad.Quad{}
return quad.Quad{}
}
err = json.Unmarshal(b, &triple)
if err != nil {
glog.Error("Error: couldn't reconstruct triple.")
return &quad.Quad{}
return quad.Quad{}
}
return &triple
return triple
}
func (qs *TripleStore) convertStringToByteHash(s string) []byte {
@ -334,7 +340,7 @@ func (qs *TripleStore) convertStringToByteHash(s string) []byte {
}
func (qs *TripleStore) ValueOf(s string) graph.Value {
return qs.createValueKeyFor(s)
return Token(qs.createValueKeyFor(s))
}
func (qs *TripleStore) valueData(value_key []byte) ValueData {
@ -362,14 +368,14 @@ func (qs *TripleStore) NameOf(k graph.Value) string {
glog.V(2).Info("k was nil")
return ""
}
return qs.valueData(k.([]byte)).Name
return qs.valueData(k.(Token)).Name
}
func (qs *TripleStore) SizeOf(k graph.Value) int64 {
if k == nil {
return 0
}
return int64(qs.valueData(k.([]byte)).Size)
return int64(qs.valueData(k.(Token)).Size)
}
func (qs *TripleStore) getSize() {
@ -432,17 +438,17 @@ func (qs *TripleStore) TriplesAllIterator() graph.Iterator {
}
func (qs *TripleStore) TripleDirection(val graph.Value, d quad.Direction) graph.Value {
v := val.([]uint8)
v := val.(Token)
offset := PositionOf(v[0:2], d, qs)
if offset != -1 {
return append([]byte("z"), v[offset:offset+qs.hasher.Size()]...)
return Token(append([]byte("z"), v[offset:offset+qs.hasher.Size()]...))
} else {
return qs.Quad(val).Get(d)
return Token(qs.Quad(val).Get(d))
}
}
func compareBytes(a, b graph.Value) bool {
return bytes.Equal(a.([]uint8), b.([]uint8))
return bytes.Equal(a.(Token), b.(Token))
}
func (qs *TripleStore) FixedIterator() graph.FixedIterator {

View file

@ -101,13 +101,13 @@ func newTripleStore() *TripleStore {
return &ts
}
func (ts *TripleStore) AddTripleSet(triples []*quad.Quad) {
func (ts *TripleStore) AddTripleSet(triples []quad.Quad) {
for _, t := range triples {
ts.AddTriple(t)
}
}
func (ts *TripleStore) tripleExists(t *quad.Quad) (bool, int64) {
func (ts *TripleStore) tripleExists(t quad.Quad) (bool, int64) {
smallest := -1
var smallest_tree *llrb.LLRB
for d := quad.Subject; d <= quad.Label; d++ {
@ -134,19 +134,19 @@ func (ts *TripleStore) tripleExists(t *quad.Quad) (bool, int64) {
for it.Next() {
val := it.Result()
if t.Equals(&ts.triples[val.(int64)]) {
if t == ts.triples[val.(int64)] {
return true, val.(int64)
}
}
return false, 0
}
func (ts *TripleStore) AddTriple(t *quad.Quad) {
func (ts *TripleStore) AddTriple(t quad.Quad) {
if exists, _ := ts.tripleExists(t); exists {
return
}
var tripleID int64
ts.triples = append(ts.triples, *t)
ts.triples = append(ts.triples, t)
tripleID = ts.tripleIdCounter
ts.size++
ts.tripleIdCounter++
@ -175,7 +175,7 @@ func (ts *TripleStore) AddTriple(t *quad.Quad) {
// TODO(barakmich): Add VIP indexing
}
func (ts *TripleStore) RemoveTriple(t *quad.Quad) {
func (ts *TripleStore) RemoveTriple(t quad.Quad) {
var tripleID int64
var exists bool
tripleID = 0
@ -221,8 +221,8 @@ func (ts *TripleStore) RemoveTriple(t *quad.Quad) {
}
}
func (ts *TripleStore) Quad(index graph.Value) *quad.Quad {
return &ts.triples[index.(int64)]
func (ts *TripleStore) Quad(index graph.Value) quad.Quad {
return ts.triples[index.(int64)]
}
func (ts *TripleStore) TripleIterator(d quad.Direction, value graph.Value) graph.Iterator {

View file

@ -37,7 +37,7 @@ import (
// \-->|#D#|------------->+---+
// +---+
//
var simpleGraph = []*quad.Quad{
var simpleGraph = []quad.Quad{
{"A", "follows", "B", ""},
{"C", "follows", "B", ""},
{"C", "follows", "D", ""},
@ -51,7 +51,7 @@ var simpleGraph = []*quad.Quad{
{"G", "status", "cool", "status_graph"},
}
func makeTestStore(data []*quad.Quad) (*TripleStore, []pair) {
func makeTestStore(data []quad.Quad) (*TripleStore, []pair) {
seen := make(map[string]struct{})
ts := newTripleStore()
var (
@ -174,7 +174,7 @@ func TestLinksToOptimization(t *testing.T) {
func TestRemoveTriple(t *testing.T) {
ts, _ := makeTestStore(simpleGraph)
ts.RemoveTriple(&quad.Quad{"E", "follows", "F", ""})
ts.RemoveTriple(quad.Quad{"E", "follows", "F", ""})
fixed := ts.FixedIterator()
fixed.Add(ts.ValueOf("E"))

View file

@ -91,7 +91,7 @@ func newTripleStore(addr string, options graph.Options) (graph.TripleStore, erro
return &qs, nil
}
func (qs *TripleStore) getIdForTriple(t *quad.Quad) string {
func (qs *TripleStore) getIdForTriple(t quad.Quad) string {
id := qs.ConvertStringToByteHash(t.Subject)
id += qs.ConvertStringToByteHash(t.Predicate)
id += qs.ConvertStringToByteHash(t.Object)
@ -150,7 +150,7 @@ func (qs *TripleStore) updateNodeBy(node_name string, inc int) {
}
}
func (qs *TripleStore) writeTriple(t *quad.Quad) bool {
func (qs *TripleStore) writeTriple(t quad.Quad) bool {
tripledoc := bson.M{
"_id": qs.getIdForTriple(t),
"Subject": t.Subject,
@ -170,7 +170,7 @@ func (qs *TripleStore) writeTriple(t *quad.Quad) bool {
return true
}
func (qs *TripleStore) AddTriple(t *quad.Quad) {
func (qs *TripleStore) AddTriple(t quad.Quad) {
_ = qs.writeTriple(t)
qs.updateNodeBy(t.Subject, 1)
qs.updateNodeBy(t.Predicate, 1)
@ -180,7 +180,7 @@ func (qs *TripleStore) AddTriple(t *quad.Quad) {
}
}
func (qs *TripleStore) AddTripleSet(in []*quad.Quad) {
func (qs *TripleStore) AddTripleSet(in []quad.Quad) {
qs.session.SetSafe(nil)
ids := make(map[string]int)
for _, t := range in {
@ -200,7 +200,7 @@ func (qs *TripleStore) AddTripleSet(in []*quad.Quad) {
qs.session.SetSafe(&mgo.Safe{})
}
func (qs *TripleStore) RemoveTriple(t *quad.Quad) {
func (qs *TripleStore) RemoveTriple(t quad.Quad) {
err := qs.db.C("triples").RemoveId(qs.getIdForTriple(t))
if err == mgo.ErrNotFound {
return
@ -216,13 +216,13 @@ func (qs *TripleStore) RemoveTriple(t *quad.Quad) {
}
}
func (qs *TripleStore) Quad(val graph.Value) *quad.Quad {
func (qs *TripleStore) Quad(val graph.Value) quad.Quad {
var bsonDoc bson.M
err := qs.db.C("triples").FindId(val.(string)).One(&bsonDoc)
if err != nil {
glog.Errorf("Error: Couldn't retrieve triple %s %v", val, err)
}
return &quad.Quad{
return quad.Quad{
bsonDoc["Subject"].(string),
bsonDoc["Predicate"].(string),
bsonDoc["Object"].(string),

View file

@ -28,29 +28,32 @@ import (
"github.com/google/cayley/quad"
)
// Defines an opaque "triple store value" type. However the backend wishes to
// implement it, a Value is merely a token to a triple or a node that the backing
// store itself understands, and the base iterators pass around.
// Value defines an opaque "triple store value" type. However the backend wishes
// to implement it, a Value is merely a token to a triple or a node that the
// backing store itself understands, and the base iterators pass around.
//
// For example, in a very traditional, graphd-style graph, these are int64s
// (guids of the primitives). In a very direct sort of graph, these could be
// pointers to structs, or merely triples, or whatever works best for the
// backing store.
//
// These must be comparable, or implement a `Key() interface{}` function
// so that they may be stored in maps.
type Value interface{}
type TripleStore interface {
// Add a triple to the store.
AddTriple(*quad.Quad)
AddTriple(quad.Quad)
// Add a set of triples to the store, atomically if possible.
AddTripleSet([]*quad.Quad)
AddTripleSet([]quad.Quad)
// Removes a triple matching the given one from the database,
// if it exists. Does nothing otherwise.
RemoveTriple(*quad.Quad)
RemoveTriple(quad.Quad)
// Given an opaque token, returns the triple for that token from the store.
Quad(Value) *quad.Quad
Quad(Value) quad.Quad
// Given a direction and a token, creates an iterator of links which have
// that node token in that directional field.