diff --git a/compact_log_test.go b/compact_log_test.go index 51ed8e47e..ea211a749 100644 --- a/compact_log_test.go +++ b/compact_log_test.go @@ -138,7 +138,7 @@ func buildTable(t *testing.T, keyValues [][]string) *os.File { }) for i, kv := range keyValues { y.AssertTrue(len(kv) == 2) - err := b.Add([]byte(kv[0]), y.ValueStruct{[]byte(kv[1]), 'A', uint16(i), 0}) + err := b.Add([]byte(kv[0]), y.ValueStruct{[]byte(kv[1]), 'A', 0, uint64(i)}) if t != nil { require.NoError(t, err) } else { diff --git a/iterator.go b/iterator.go index 1ca576daf..8a18c36b5 100644 --- a/iterator.go +++ b/iterator.go @@ -32,7 +32,7 @@ type KVItem struct { meta byte userMeta byte val []byte - casCounter uint16 + casCounter uint64 slice *y.Slice next *KVItem } @@ -51,7 +51,7 @@ func (item *KVItem) Value() []byte { } // Counter returns the CAS counter associated with the value. -func (item *KVItem) Counter() uint16 { +func (item *KVItem) Counter() uint64 { return item.casCounter } diff --git a/kv.go b/kv.go index eb378f223..6b6e7f83d 100644 --- a/kv.go +++ b/kv.go @@ -22,6 +22,7 @@ import ( "os" "path/filepath" "sync" + "sync/atomic" "time" "golang.org/x/net/trace" @@ -112,10 +113,9 @@ var DefaultOptions = Options{ func (opt *Options) estimateSize(entry *Entry) int { if len(entry.Value) < opt.ValueThreshold { - // 4 is for cas + meta - return len(entry.Key) + len(entry.Value) + 4 + return len(entry.Key) + len(entry.Value) + y.MetaSize + y.UserMetaSize + y.CasSize } - return len(entry.Key) + 16 + 4 + return len(entry.Key) + 16 + y.MetaSize + y.UserMetaSize + y.CasSize } // KV provides the various functions required to interact with Badger. @@ -137,6 +137,10 @@ type KV struct { vptr valuePointer writeCh chan *request flushChan chan flushTask // For flushing memtables. + + // Incremented in the non-concurrently accessed write loop. But also accessed outside. So + // we use an atomic op. + lastUsedCasCounter uint64 } var ErrInvalidDir error = errors.New("Invalid Dir, directory does not exist") @@ -223,6 +227,11 @@ func NewKV(opt *Options) (out *KV, err error) { return nil, errors.Wrap(err, "Retrieving head") } val := item.Value() + // lastUsedCasCounter will either be the value stored in !badger!head, or some subsequently + // written value log entry that we replay. (Subsequent value log entries might be _less_ + // than lastUsedCasCounter, if there was value log gc so we have to max() values while + // replaying.) + out.lastUsedCasCounter = item.casCounter var vptr valuePointer if len(val) > 0 { @@ -238,6 +247,9 @@ func NewKV(opt *Options) (out *KV, err error) { out.elog.Printf("First key=%s\n", e.Key) } first = false + if out.lastUsedCasCounter < e.casCounter { + out.lastUsedCasCounter = e.casCounter + } if e.CASCounterCheck != 0 { oldValue, err := out.get(e.Key) @@ -584,6 +596,18 @@ func (s *KV) writeToLSM(b *request) error { return nil } +// lastCASCounter returns the last-used cas counter. +func (s *KV) lastCASCounter() uint64 { + return atomic.LoadUint64(&s.lastUsedCasCounter) +} + +// newCASCounters generates a set of unique CAS counters -- the interval [x, x + howMany) where x +// is the return value. +func (s *KV) newCASCounters(howMany uint64) uint64 { + last := atomic.AddUint64(&s.lastUsedCasCounter, howMany) + return last - howMany + 1 +} + // writeRequests is called serially by only one goroutine. func (s *KV) writeRequests(reqs []*request) error { if len(reqs) == 0 { @@ -599,12 +623,17 @@ func (s *KV) writeRequests(reqs []*request) error { s.elog.Printf("writeRequests called. Writing to value log") - // CAS counter for all operations has to go onto value log. Otherwise, if it is just in memtable for - // a long time, and following CAS operations use that as a check, when replaying, we will think that - // these CAS operations should fail, when they are actually valid. + // CAS counter for all operations has to go onto value log. Otherwise, if it is just in + // memtable for a long time, and following CAS operations use that as a check, when + // replaying, we will think that these CAS operations should fail, when they are actually + // valid. + + // There is code (in flushMemtable) whose correctness depends on us generating CAS Counter + // values _before_ we modify s.vptr here. for _, req := range reqs { - for _, e := range req.Entries { - e.casCounter = newCASCounter() + counterBase := s.newCASCounters(uint64(len(req.Entries))) + for i, e := range req.Entries { + e.casCounter = counterBase + uint64(i) } } err := s.vlog.write(reqs) @@ -816,7 +845,7 @@ func EntriesSet(s []*Entry, key, val []byte) []*Entry { // CompareAndSet sets the given value, ensuring that the no other Set operation has happened, // since last read. If the key has a different casCounter, this would not update the key // and return an error. -func (s *KV) CompareAndSet(key []byte, val []byte, casCounter uint16) error { +func (s *KV) CompareAndSet(key []byte, val []byte, casCounter uint64) error { e := &Entry{ Key: key, Value: val, @@ -847,7 +876,7 @@ func (s *KV) compareAsync(e *Entry, f func(error)) { // CompareAndSetAsync is the asynchronous version of CompareAndSet. It accepts a callback function // which is called when the CompareAndSet completes. Any error encountered during execution is // passed as an argument to the callback function. -func (s *KV) CompareAndSetAsync(key []byte, val []byte, casCounter uint16, f func(error)) { +func (s *KV) CompareAndSetAsync(key []byte, val []byte, casCounter uint64, f func(error)) { e := &Entry{ Key: key, Value: val, @@ -889,7 +918,7 @@ func EntriesDelete(s []*Entry, key []byte) []*Entry { // CompareAndDelete deletes a key ensuring that it has not been changed since last read. // If existing key has different casCounter, this would not delete the key and return an error. -func (s *KV) CompareAndDelete(key []byte, casCounter uint16) error { +func (s *KV) CompareAndDelete(key []byte, casCounter uint64) error { e := &Entry{ Key: key, Meta: BitDelete, @@ -904,7 +933,7 @@ func (s *KV) CompareAndDelete(key []byte, casCounter uint16) error { // CompareAndDeleteAsync is the asynchronous version of CompareAndDelete. It accepts a callback // function which is called when the CompareAndDelete completes. Any error encountered during // execution is passed as an argument to the callback function. -func (s *KV) CompareAndDeleteAsync(key []byte, casCounter uint16, f func(error)) { +func (s *KV) CompareAndDeleteAsync(key []byte, casCounter uint64, f func(error)) { e := &Entry{ Key: key, Meta: BitDelete, @@ -983,10 +1012,18 @@ func (s *KV) flushMemtable(lc *y.LevelCloser) error { if ft.vptr.Fid > 0 || ft.vptr.Offset > 0 || ft.vptr.Len > 0 { s.elog.Printf("Storing offset: %+v\n", ft.vptr) offset := make([]byte, 10) - s.Lock() // For vptr. + s.Lock() // For vptr and casAsOfVptr. s.vptr.Encode(offset) s.Unlock() - ft.mt.Put(head, y.ValueStruct{Value: offset}) // casCounter not needed. + // CAS counter is needed and is desirable -- it's the first value log entry + // we replay, so to speak, perhaps the only, and we use it to re-initialize + // the CAS counter. + // + // The write loop generates CAS counter values _before_ it sets vptr. It + // is crucial that we read the cas counter here _after_ reading vptr. That + // way, our value here is guaranteed to be >= the CASCounter values written + // before vptr (because they don't get replayed). + ft.mt.Put(head, y.ValueStruct{Value: offset, CASCounter: s.lastCASCounter()}) } fileID, _ := s.lc.reserveFileIDs(1) fd, err := y.OpenSyncedFile(table.NewFilename(fileID, s.opt.Dir), true) diff --git a/skl/arena.go b/skl/arena.go index 012066287..978640598 100644 --- a/skl/arena.go +++ b/skl/arena.go @@ -17,7 +17,6 @@ package skl import ( - "encoding/binary" "sync/atomic" "github.com/dgraph-io/badger/y" @@ -50,16 +49,13 @@ func (s *Arena) Reset() { // size of val. We could also store this size inside arena but the encoding and // decoding will incur some overhead. func (s *Arena) PutVal(v y.ValueStruct) uint32 { - l := uint32(len(v.Value)) + 4 + l := uint32(v.EncodedSize()) n := atomic.AddUint32(&s.n, l) y.AssertTruef(int(n) <= len(s.buf), "Arena too small, toWrite:%d newTotal:%d limit:%d", l, n, len(s.buf)) m := n - l - s.buf[m] = v.Meta - s.buf[m+1] = v.UserMeta - binary.BigEndian.PutUint16(s.buf[m+2:m+4], v.CASCounter) - copy(s.buf[m+4:n], v.Value) + v.Encode(s.buf[m:]) return m } @@ -80,13 +76,8 @@ func (s *Arena) GetKey(offset uint32, size uint16) []byte { } // GetVal returns byte slice at offset. The given size should be just the value -// size and should NOT include the meta byte. -func (s *Arena) GetVal(offset uint32, size uint16) y.ValueStruct { - out := y.ValueStruct{ - Value: s.buf[offset+4 : offset+4+uint32(size)], - Meta: s.buf[offset], - UserMeta: s.buf[offset+1], - CASCounter: binary.BigEndian.Uint16(s.buf[offset+2 : offset+4]), - } - return out +// size and should NOT include the meta bytes. +func (s *Arena) GetVal(offset uint32, size uint16) (ret y.ValueStruct) { + ret.DecodeEntireSlice(s.buf[offset : offset+uint32(y.ValueStructSerializedSize(size))]) + return } diff --git a/skl/skl_test.go b/skl/skl_test.go index 43759521b..83c6fa6a1 100644 --- a/skl/skl_test.go +++ b/skl/skl_test.go @@ -91,9 +91,9 @@ func TestBasic(t *testing.T) { // Try inserting values. // Somehow require.Nil doesn't work when checking for unsafe.Pointer(nil). - l.Put([]byte("key1"), y.ValueStruct{val1, 55, 60000, 0}) - l.Put([]byte("key3"), y.ValueStruct{val3, 56, 60001, 0}) - l.Put([]byte("key2"), y.ValueStruct{val2, 57, 60002, 0}) + l.Put([]byte("key1"), y.ValueStruct{val1, 55, 0, 60000}) + l.Put([]byte("key3"), y.ValueStruct{val3, 56, 0, 60001}) + l.Put([]byte("key2"), y.ValueStruct{val2, 57, 0, 60002}) v := l.Get([]byte("key")) require.True(t, v.Value == nil) @@ -116,7 +116,7 @@ func TestBasic(t *testing.T) { require.EqualValues(t, 56, v.Meta) require.EqualValues(t, 60001, v.CASCounter) - l.Put([]byte("key2"), y.ValueStruct{val4, 12, 50000, 0}) + l.Put([]byte("key2"), y.ValueStruct{val4, 12, 0, 50000}) v = l.Get([]byte("key2")) require.True(t, v.Value != nil) require.EqualValues(t, "00072", string(v.Value)) @@ -134,7 +134,7 @@ func TestConcurrentBasic(t *testing.T) { go func(i int) { defer wg.Done() l.Put([]byte(fmt.Sprintf("%05d", i)), - y.ValueStruct{newValue(i), 0, uint16(i), 0}) + y.ValueStruct{newValue(i), 0, 0, uint64(i)}) }(i) } wg.Wait() @@ -165,7 +165,7 @@ func TestOneKey(t *testing.T) { wg.Add(1) go func(i int) { defer wg.Done() - l.Put(key, y.ValueStruct{newValue(i), 0, uint16(i), 0}) + l.Put(key, y.ValueStruct{newValue(i), 0, 0, uint64(i)}) }(i) } // We expect that at least some write made it such that some read returns a value. @@ -195,7 +195,7 @@ func TestFindNear(t *testing.T) { defer l.DecrRef() for i := 0; i < 1000; i++ { key := fmt.Sprintf("%05d", i*10+5) - l.Put([]byte(key), y.ValueStruct{newValue(i), 0, uint16(i), 0}) + l.Put([]byte(key), y.ValueStruct{newValue(i), 0, 0, uint64(i)}) } n, eq := l.findNear([]byte("00001"), false, false) @@ -307,7 +307,7 @@ func TestIteratorNext(t *testing.T) { require.False(t, it.Valid()) for i := n - 1; i >= 0; i-- { l.Put([]byte(fmt.Sprintf("%05d", i)), - y.ValueStruct{newValue(i), 0, uint16(i), 0}) + y.ValueStruct{newValue(i), 0, 0, uint64(i)}) } it.SeekToFirst() for i := 0; i < n; i++ { @@ -331,7 +331,7 @@ func TestIteratorPrev(t *testing.T) { require.False(t, it.Valid()) for i := 0; i < n; i++ { l.Put([]byte(fmt.Sprintf("%05d", i)), - y.ValueStruct{newValue(i), 0, uint16(i), 0}) + y.ValueStruct{newValue(i), 0, 0, uint64(i)}) } it.SeekToLast() for i := n - 1; i >= 0; i-- { @@ -359,7 +359,7 @@ func TestIteratorSeek(t *testing.T) { // 1000, 1010, 1020, ..., 1990. for i := n - 1; i >= 0; i-- { v := i*10 + 1000 - l.Put([]byte(fmt.Sprintf("%05d", i*10+1000)), y.ValueStruct{newValue(v), 0, 555, 0}) + l.Put([]byte(fmt.Sprintf("%05d", i*10+1000)), y.ValueStruct{newValue(v), 0, 0, 555}) } it.Seek([]byte("")) require.True(t, it.Valid()) diff --git a/table/builder.go b/table/builder.go index b97684de1..715eeabc6 100644 --- a/table/builder.go +++ b/table/builder.go @@ -136,8 +136,8 @@ func (b *TableBuilder) addHelper(key []byte, v y.ValueStruct) { h := header{ plen: uint16(len(key) - len(diffKey)), klen: uint16(len(diffKey)), - vlen: uint16(len(v.Value) + 1 + 2 + 1), // Include meta byte and casCounter. - prev: b.prevOffset, // prevOffset is the location of the last key-value added. + vlen: uint16(len(v.Value) + y.MetaSize + y.UserMetaSize + y.CasSize), + prev: b.prevOffset, // prevOffset is the location of the last key-value added. } b.prevOffset = uint32(b.buf.Len()) - b.baseOffset // Remember current offset for the next Add call. @@ -148,8 +148,8 @@ func (b *TableBuilder) addHelper(key []byte, v y.ValueStruct) { b.buf.Write(diffKey) // We only need to store the key difference. b.buf.WriteByte(v.Meta) // Meta byte precedes actual value. b.buf.WriteByte(v.UserMeta) - var casBytes [2]byte - binary.BigEndian.PutUint16(casBytes[:], v.CASCounter) + var casBytes [y.CasSize]byte + binary.BigEndian.PutUint64(casBytes[:], v.CASCounter) b.buf.Write(casBytes[:]) b.buf.Write(v.Value) b.counter++ // Increment number of keys added for this current block. diff --git a/table/iterator.go b/table/iterator.go index d49fc2516..d1d1ef7ec 100644 --- a/table/iterator.go +++ b/table/iterator.go @@ -18,7 +18,6 @@ package table import ( "bytes" - "encoding/binary" "io" "math" "sort" @@ -391,14 +390,9 @@ func (itr *TableIterator) Key() []byte { return itr.bi.Key() } -func (itr *TableIterator) Value() y.ValueStruct { - v := itr.bi.Value() - return y.ValueStruct{ - Value: v[4:], - Meta: v[0], - UserMeta: v[1], - CASCounter: binary.BigEndian.Uint16(v[2:4]), - } +func (itr *TableIterator) Value() (ret y.ValueStruct) { + ret.DecodeEntireSlice(itr.bi.Value()) + return } func (s *TableIterator) Next() { diff --git a/table/table_test.go b/table/table_test.go index 3555a5d19..a6b1151f1 100644 --- a/table/table_test.go +++ b/table/table_test.go @@ -61,7 +61,7 @@ func buildTable(t *testing.T, keyValues [][]string) *os.File { }) for i, kv := range keyValues { y.AssertTrue(len(kv) == 2) - err := b.Add([]byte(kv[0]), y.ValueStruct{[]byte(kv[1]), 'A', uint16(i), 0}) + err := b.Add([]byte(kv[0]), y.ValueStruct{[]byte(kv[1]), 'A', 0, uint64(i)}) if t != nil { require.NoError(t, err) } else { @@ -617,7 +617,7 @@ func BenchmarkRead(b *testing.B) { for i := 0; i < n; i++ { k := fmt.Sprintf("%016x", i) v := fmt.Sprintf("%d", i) - y.Check(builder.Add([]byte(k), y.ValueStruct{[]byte(v), 123, 5555, 0})) + y.Check(builder.Add([]byte(k), y.ValueStruct{[]byte(v), 123, 0, 5555})) } f.Write(builder.Finish([]byte("somemetadata"))) @@ -647,7 +647,7 @@ func BenchmarkReadAndBuild(b *testing.B) { for i := 0; i < n; i++ { k := fmt.Sprintf("%016x", i) v := fmt.Sprintf("%d", i) - y.Check(builder.Add([]byte(k), y.ValueStruct{[]byte(v), 123, 5555, 0})) + y.Check(builder.Add([]byte(k), y.ValueStruct{[]byte(v), 123, 0, 5555})) } f.Write(builder.Finish([]byte("somemetadata"))) @@ -687,7 +687,7 @@ func BenchmarkReadMerged(b *testing.B) { // id := i*tableSize+j (not interleaved) k := fmt.Sprintf("%016x", id) v := fmt.Sprintf("%d", id) - y.Check(builder.Add([]byte(k), y.ValueStruct{[]byte(v), 123, 5555, 0})) + y.Check(builder.Add([]byte(k), y.ValueStruct{[]byte(v), 123, 0, 5555})) } f.Write(builder.Finish([]byte("somemetadata"))) tbl, err := OpenTable(f, MemoryMap) diff --git a/util.go b/util.go index cea9fc33b..8cfe82fd8 100644 --- a/util.go +++ b/util.go @@ -136,22 +136,6 @@ func getIDMap(dir string) map[uint64]struct{} { return idMap } -// mod65535 mods by 65535 fast. -func mod65535(a uint32) uint32 { - a = (a >> 16) + (a & 0xFFFF) /* sum base 2**16 digits */ - if a < 65535 { - return a - } - if a < (2 * 65535) { - return a - 65535 - } - return a - (2 * 65535) -} - -func newCASCounter() uint16 { - return uint16(1 + mod65535(rand.Uint32())) -} - func init() { rand.Seed(time.Now().UnixNano()) } diff --git a/value.go b/value.go index 6de6c0953..0274da12e 100644 --- a/value.go +++ b/value.go @@ -110,7 +110,7 @@ type logEntry func(e Entry, vp valuePointer) error // iterate iterates over log file. It doesn't not allocate new memory for every kv pair. // Therefore, the kv pair is only valid for the duration of fn call. func (f *logFile) iterate(offset uint32, fn logEntry) error { - _, err := f.fd.Seek(int64(offset), 0) + _, err := f.fd.Seek(int64(offset), io.SeekStart) if err != nil { return y.Wrap(err) } @@ -129,7 +129,7 @@ func (f *logFile) iterate(offset uint32, fn logEntry) error { } reader := bufio.NewReader(f.fd) - var hbuf [14]byte + var hbuf [headerBufSize]byte var h header var count int k := make([]byte, 1<<10) @@ -324,12 +324,12 @@ type Entry struct { Meta byte UserMeta byte Value []byte - CASCounterCheck uint16 // If nonzero, we will check if existing casCounter matches. + CASCounterCheck uint64 // If nonzero, we will check if existing casCounter matches. Error error // Error if any. // Fields maintained internally. offset uint32 - casCounter uint16 + casCounter uint64 } type entryEncoder struct { @@ -341,7 +341,7 @@ type entryEncoder struct { // Encodes e to buf either plain or compressed. // Returns number of bytes written. func (enc *entryEncoder) Encode(e *Entry, buf *bytes.Buffer) (int, error) { - var headerEnc [14]byte + var headerEnc [headerBufSize]byte var h header if int32(len(e.Key)+len(e.Value)) > enc.opt.ValueCompressionMinSize { @@ -396,18 +396,22 @@ type header struct { vlen uint32 // len of value or length of compressed kv if entry stored compressed meta byte userMeta byte - casCounter uint16 - casCounterCheck uint16 + casCounter uint64 + casCounterCheck uint64 } +const ( + headerBufSize = 26 +) + func (h header) Encode(out []byte) { - y.AssertTrue(len(out) >= 14) + y.AssertTrue(len(out) >= headerBufSize) binary.BigEndian.PutUint32(out[0:4], h.klen) binary.BigEndian.PutUint32(out[4:8], h.vlen) out[8] = h.meta out[9] = h.userMeta - binary.BigEndian.PutUint16(out[10:12], h.casCounter) - binary.BigEndian.PutUint16(out[12:14], h.casCounterCheck) + binary.BigEndian.PutUint64(out[10:18], h.casCounter) + binary.BigEndian.PutUint64(out[18:26], h.casCounterCheck) } // Decodes h from buf. Returns buf without header and number of bytes read. @@ -416,9 +420,9 @@ func (h *header) Decode(buf []byte) ([]byte, int) { h.vlen = binary.BigEndian.Uint32(buf[4:8]) h.meta = buf[8] h.userMeta = buf[9] - h.casCounter = binary.BigEndian.Uint16(buf[10:12]) - h.casCounterCheck = binary.BigEndian.Uint16(buf[12:14]) - return buf[14:], 14 + h.casCounter = binary.BigEndian.Uint64(buf[10:18]) + h.casCounterCheck = binary.BigEndian.Uint64(buf[18:26]) + return buf[26:], 26 } type valuePointer struct { @@ -585,10 +589,12 @@ func (l *valueLog) Replay(ptr valuePointer, fn logEntry) error { } type request struct { + // Input values Entries []*Entry - Ptrs []valuePointer - Wg sync.WaitGroup - Err error + // Output values and wait group stuff below + Ptrs []valuePointer + Wg sync.WaitGroup + Err error } // sync is thread-unsafe and should not be called concurrently with write. diff --git a/y/iterator.go b/y/iterator.go index 9d53c4a43..d35105d57 100644 --- a/y/iterator.go +++ b/y/iterator.go @@ -19,6 +19,7 @@ package y import ( "bytes" "container/heap" + "encoding/binary" "github.com/pkg/errors" ) @@ -26,8 +27,40 @@ import ( type ValueStruct struct { Value []byte Meta byte - CASCounter uint16 UserMeta byte + CASCounter uint64 +} + +func (v *ValueStruct) EncodedSize() int { + return len(v.Value) + valueValueOffset +} + +// Converts a value size to the full serialized size of value + metadata. +func ValueStructSerializedSize(size uint16) int { + return int(size) + valueValueOffset +} + +const ( + valueMetaOffset = 0 + valueUserMetaOffset = valueMetaOffset + MetaSize + valueCasOffset = valueUserMetaOffset + UserMetaSize + valueValueOffset = valueCasOffset + CasSize +) + +// DecodeEntireSlice uses the length of the slice to infer the length of the Value field. +func (v *ValueStruct) DecodeEntireSlice(b []byte) { + v.Value = b[valueValueOffset:] + v.Meta = b[valueMetaOffset] + v.UserMeta = b[valueUserMetaOffset] + v.CASCounter = binary.BigEndian.Uint64(b[valueCasOffset : valueCasOffset+CasSize]) +} + +// Encode expects a slice of length at least v.EncodedSize(). +func (v *ValueStruct) Encode(b []byte) { + b[valueMetaOffset] = v.Meta + b[valueUserMetaOffset] = v.UserMeta + binary.BigEndian.PutUint64(b[valueCasOffset:valueCasOffset+CasSize], v.CASCounter) + copy(b[valueValueOffset:valueValueOffset+len(v.Value)], v.Value) } // Iterator is an interface for a basic iterator. diff --git a/y/iterator_test.go b/y/iterator_test.go index 08892d381..c5be06aff 100644 --- a/y/iterator_test.go +++ b/y/iterator_test.go @@ -70,7 +70,7 @@ func (s *SimpleIterator) Seek(key []byte) { func (s *SimpleIterator) Key() []byte { return s.keys[s.idx] } func (s *SimpleIterator) Value() ValueStruct { - return ValueStruct{s.vals[s.idx], 55, 12345, 0} + return ValueStruct{s.vals[s.idx], 55, 0, 12345} } func (s *SimpleIterator) Valid() bool { return s.idx >= 0 && s.idx < len(s.keys) diff --git a/y/y.go b/y/y.go index 806fcb75a..0d8bbbaef 100644 --- a/y/y.go +++ b/y/y.go @@ -23,6 +23,13 @@ import ( "sync/atomic" ) +// Constants used in serialization sizes, and in ValueStruct serialization +const ( + MetaSize = 1 + UserMetaSize = 1 + CasSize = 8 +) + var ( // This is O_DSYNC (datasync) on platforms that support it -- see file_unix.go datasyncFileFlag = 0x0