diff --git a/Makefile b/Makefile index 6f0f6c47cc..f5f5b5dd84 100644 --- a/Makefile +++ b/Makefile @@ -19,7 +19,7 @@ endif #PEG_DEP=peg vet: - @go vet -unsafeptr=false ./... + @go vet ./... fmt: gofmt -s -w . diff --git a/api/client/request.go b/api/client/request.go index 10d9f14025..25eb8caa62 100644 --- a/api/client/request.go +++ b/api/client/request.go @@ -9,7 +9,6 @@ import ( "net/http/httptrace" "time" - "github.com/brimdata/zed" "github.com/brimdata/zed/api" "github.com/brimdata/zed/zio" "github.com/brimdata/zed/zio/zngio" @@ -96,9 +95,7 @@ func (r *Request) reader() (io.Reader, error) { } m := zson.NewZNGMarshaler() m.Decorate(zson.StylePackage) - arena := zed.NewArena() - defer arena.Unref() - val, err := m.Marshal(arena, r.Body) + val, err := m.Marshal(r.Body) if err != nil { return nil, err } diff --git a/api/queryio/client.go b/api/queryio/client.go index 2004b8b627..df99797bf8 100644 --- a/api/queryio/client.go +++ b/api/queryio/client.go @@ -77,9 +77,7 @@ func marshalControl(zctrl *zbuf.Control) (any, error) { if ctrl.Format != zngio.ControlFormatZSON { return nil, fmt.Errorf("unsupported app encoding: %v", ctrl.Format) } - arena := zed.NewArena() - defer arena.Unref() - value, err := zson.ParseValue(zed.NewContext(), arena, string(ctrl.Bytes)) + value, err := zson.ParseValue(zed.NewContext(), string(ctrl.Bytes)) if err != nil { return nil, fmt.Errorf("unable to parse Zed control message: %w (%s)", err, ctrl.Bytes) } diff --git a/api/queryio/zjson_test.go b/api/queryio/zjson_test.go index 6acc105500..17bc8cec60 100644 --- a/api/queryio/zjson_test.go +++ b/api/queryio/zjson_test.go @@ -24,9 +24,7 @@ func TestZJSONWriter(t *testing.T) { w := queryio.NewZJSONWriter(&buf) err := w.WriteControl(api.QueryChannelSet{Channel: "main"}) require.NoError(t, err) - arena := zed.NewArena() - defer arena.Unref() - err = w.Write(zson.MustParseValue(zed.NewContext(), arena, record)) + err = w.Write(zson.MustParseValue(zed.NewContext(), record)) require.NoError(t, err) err = w.WriteControl(api.QueryChannelEnd{Channel: "main"}) require.NoError(t, err) diff --git a/api/queryio/zng.go b/api/queryio/zng.go index 0b80132ac4..1b24c9a929 100644 --- a/api/queryio/zng.go +++ b/api/queryio/zng.go @@ -4,7 +4,6 @@ import ( "bytes" "io" - "github.com/brimdata/zed" "github.com/brimdata/zed/zio" "github.com/brimdata/zed/zio/zngio" "github.com/brimdata/zed/zio/zsonio" @@ -14,7 +13,6 @@ import ( type ZNGWriter struct { *zngio.Writer marshaler *zson.MarshalZNGContext - arena *zed.Arena } var _ controlWriter = (*ZJSONWriter)(nil) @@ -25,13 +23,11 @@ func NewZNGWriter(w io.Writer) *ZNGWriter { return &ZNGWriter{ Writer: zngio.NewWriter(zio.NopCloser(w)), marshaler: m, - arena: zed.NewArena(), } } func (w *ZNGWriter) WriteControl(v interface{}) error { - w.arena.Reset() - val, err := w.marshaler.Marshal(w.arena, v) + val, err := w.marshaler.Marshal(v) if err != nil { return err } diff --git a/arena.go b/arena.go deleted file mode 100644 index aef1ef21a2..0000000000 --- a/arena.go +++ /dev/null @@ -1,219 +0,0 @@ -package zed - -import ( - "encoding/binary" - "math" - "net/netip" - "slices" - "strconv" - "sync" - "sync/atomic" - "unsafe" - - "github.com/brimdata/zed/zcode" -) - -// Arena is an reference-counted allocator for Values. Two constraints govern -// its use. First, the Type of each Value in an arena must belong to a single -// Context. Second, an arena must be reachable at any point in a program where -// its Values are accessed. -type Arena struct { - pool *sync.Pool - refs int32 - - byID []Type - offsetsAndLengths []uint64 - - buf []byte - free func() - slices [][]byte - values []Value -} - -var ( - arenaPool sync.Pool - arenaWithBufferPool sync.Pool -) - -// NewArena returns an empty arena. -func NewArena() *Arena { - return newArena(&arenaPool) -} - -// NewArenaWithBuffer returns an arena whose buffer is initialized to buf. If -// free is not nil, it is called when Unref decrements the arena's reference -// count to zero and can be used to return buf to an allocator. -func NewArenaWithBuffer(buf []byte, free func()) *Arena { - a := newArena(&arenaWithBufferPool) - a.buf = buf - a.free = free - return a -} - -func newArena(pool *sync.Pool) *Arena { - a, ok := pool.Get().(*Arena) - if ok { - clear(a.byID) - a.Reset() - } else { - a = &Arena{pool: pool} - } - a.refs = 1 - if a.buf == nil { - a.buf = []byte{} - } - return a -} - -func (a *Arena) Ref() { - atomic.AddInt32(&a.refs, 1) -} - -func (a *Arena) Unref() { - if refs := atomic.AddInt32(&a.refs, -1); refs == 0 { - if a.free != nil { - a.buf = nil - a.free() - } - a.pool.Put(a) - } else if refs < 0 { - panic("negative arena reference count") - } -} - -func (a *Arena) Reset() { - a.offsetsAndLengths = a.offsetsAndLengths[:0] - a.buf = a.buf[:0] - a.slices = a.slices[:0] - a.values = a.values[:0] -} - -// New returns a new Value in a whose bytes are given by b. a retains a -// reference to b, so modifying b also modifies the value. -func (a *Arena) New(t Type, b zcode.Bytes) Value { - if b == nil { - return a.new(t, dStorageNull, 0, 0) - } - offset := len(a.slices) - a.slices = append(a.slices, b) - return a.new(t, dStorageSlices, offset, 0) -} - -// NewFromOffsetAndLength returns a new Value in a whose bytes are a slice of -// a's buffer from offset to offet+length. NewFromOffsetAndLength is intended -// for use with arenas obtained from NewArenaWithBuffer. -func (a *Arena) NewFromOffsetAndLength(t Type, offset, length int) Value { - return a.new(t, dStorageBuffer, offset, length) -} - -// NewFromValues returns a new Value in a that is a record, array, set, map, -// union, or error with values as its constituent values. NewFromValues does -// not retain a reference to values. -func (a *Arena) NewFromValues(t Type, values []Value) Value { - if values == nil { - return a.new(t, dStorageNull, 0, 0) - } - offset := len(a.values) - a.values = append(a.values, values...) - return a.new(t, dStorageValues, offset, len(values)) -} - -func (a *Arena) new(t Type, dStorage uint64, offset, length int) Value { - id := TypeID(t) - if id >= len(a.byID) { - a.byID = slices.Grow(a.byID[:0], id+1)[:id+1] - } - if tt := a.byID[id]; tt == nil { - a.byID[id] = t - } else if tt != t { - panic("type conflict") - } - var slot int - if dStorage != dStorageNull { - if uint64(offset) > math.MaxUint32 { - panic("bad offset " + strconv.Itoa(offset)) - } - if uint64(length) > math.MaxUint32 { - panic("bad length " + strconv.Itoa(length)) - } - a.offsetsAndLengths = append(a.offsetsAndLengths, uint64(offset)<<32|uint64(length)) - slot = len(a.offsetsAndLengths) - 1 - } - return Value{aReprArena | uint64(uintptr(unsafe.Pointer(a))), dStorage | uint64(id)<<32 | uint64(slot)} -} - -// NewBytes returns a new Value in a of type TypeBytes. a may retain a -// reference to b, so modifying b may also modify the value. -func (a *Arena) NewBytes(b []byte) Value { - if len(b) < 16 { - if b == nil { - return NullBytes - } - return newSmallBytesOrString(aReprSmallBytes, b) - } - return a.New(TypeBytes, b) -} - -// NewString returns a new Value in a of type TypeString. -func (a *Arena) NewString(s string) Value { - if len(s) < 16 { - return newSmallBytesOrString(aReprsSmallString, []byte(s)) - } - return a.New(TypeString, []byte(s)) -} - -func newSmallBytesOrString(aRepr uint64, b []byte) Value { - var bb [16]byte - copy(bb[1:], b) - a := binary.BigEndian.Uint64(bb[:]) - d := binary.BigEndian.Uint64(bb[8:]) - return Value{aRepr | uint64(len(b))<<56 | a, d} -} - -// NewIP returns a new Value in a of type TypeIP. -func (a *Arena) NewIP(A netip.Addr) Value { - return a.New(TypeIP, EncodeIP(A)) -} - -// NewNet returns a new Value in a of type TypeNet. -func (a *Arena) NewNet(p netip.Prefix) Value { - return a.New(TypeNet, EncodeNet(p)) -} - -func (a *Arena) type_(d uint64) Type { - return a.byID[d&^dStorageMask>>32] -} - -func (a *Arena) bytes_(d uint64) zcode.Bytes { - switch d & dStorageMask { - case dStorageBuffer: - offset, length := a.offsetAndLength(d) - return a.buf[offset : offset+length] - case dStorageNull: - return nil - case dStorageSlices: - offset, _ := a.offsetAndLength(d) - return a.slices[offset] - case dStorageValues: - offset, length := a.offsetAndLength(d) - if union, ok := TypeUnder(a.type_(d)).(*TypeUnion); ok { - val := a.values[offset] - tag := union.TagOf(val.Type()) - b := zcode.Append(nil, EncodeInt(int64(tag))) - return zcode.Append(b, val.Bytes()) - } - b := zcode.Bytes{} - for _, val := range a.values[offset : offset+length] { - b = zcode.Append(b, val.Bytes()) - } - return b - } - panic(d) -} - -func (a *Arena) offsetAndLength(d uint64) (uint64, uint64) { - slot := d & math.MaxUint32 - offset := a.offsetsAndLengths[slot] >> 32 - length := a.offsetsAndLengths[slot] & 0xffffffff - return offset, length -} diff --git a/cli/zq/command.go b/cli/zq/command.go index 48be47d307..60c2b2120c 100644 --- a/cli/zq/command.go +++ b/cli/zq/command.go @@ -143,7 +143,7 @@ func (c *Command) Run(args []string) error { local := storage.NewLocalEngine() var readers []zio.Reader if null { - readers = []zio.Reader{zbuf.NewArray(nil, []zed.Value{zed.Null})} + readers = []zio.Reader{zbuf.NewArray([]zed.Value{zed.Null})} } else { readers, err = c.inputFlags.Open(ctx, zctx, local, paths, c.stopErr) if err != nil { diff --git a/cmd/zed/dev/dig/frames/command.go b/cmd/zed/dev/dig/frames/command.go index 96e86db84a..7f248721c1 100644 --- a/cmd/zed/dev/dig/frames/command.go +++ b/cmd/zed/dev/dig/frames/command.go @@ -77,7 +77,6 @@ func (c *Command) Run(args []string) error { type metaReader struct { reader *reader marshaler *zson.MarshalZNGContext - arena *zed.Arena } var _ zio.Reader = (*metaReader)(nil) @@ -86,7 +85,6 @@ func newMetaReader(r io.Reader) *metaReader { return &metaReader{ reader: &reader{reader: bufio.NewReader(r)}, marshaler: zson.NewZNGMarshaler(), - arena: zed.NewArena(), } } @@ -118,8 +116,7 @@ func (m *metaReader) Read() (*zed.Value, error) { if f == nil || err != nil { return nil, err } - m.arena.Reset() - val, err := m.marshaler.Marshal(m.arena, f) + val, err := m.marshaler.Marshal(f) return &val, err } diff --git a/cmd/zed/dev/vector/agg/command.go b/cmd/zed/dev/vector/agg/command.go index e6d05e1dbd..fd167c1b07 100644 --- a/cmd/zed/dev/vector/agg/command.go +++ b/cmd/zed/dev/vector/agg/command.go @@ -62,16 +62,15 @@ func (c *Command) Run(args []string) error { return err } field := args[1] - zctx := zed.NewContext() local := storage.NewLocalEngine() cache := vcache.NewCache(local) - object, err := cache.Fetch(ctx, zctx, uri, ksuid.Nil) + object, err := cache.Fetch(ctx, uri, ksuid.Nil) if err != nil { return err } defer object.Close() //XXX nil puller - agg := op.NewCountByString(zctx, nil, field) + agg := op.NewCountByString(zed.NewContext(), nil, field) writer, err := c.outputFlags.Open(ctx, local) if err != nil { return err diff --git a/cmd/zed/dev/vector/copy/command.go b/cmd/zed/dev/vector/copy/command.go index 9c192e7edf..a8b5633525 100644 --- a/cmd/zed/dev/vector/copy/command.go +++ b/cmd/zed/dev/vector/copy/command.go @@ -57,9 +57,8 @@ func (c *Command) Run(args []string) error { if err != nil { return err } - zctx := zed.NewContext() local := storage.NewLocalEngine() - object, err := vcache.NewObject(ctx, zctx, local, uri) + object, err := vcache.NewObject(ctx, local, uri) if err != nil { return err } @@ -68,7 +67,7 @@ func (c *Command) Run(args []string) error { if err != nil { return err } - puller := vam.NewProjection(zctx, object, nil) + puller := vam.NewProjection(zed.NewContext(), object, nil) if err := zbuf.CopyPuller(writer, puller); err != nil { writer.Close() return err diff --git a/cmd/zed/dev/vector/project/command.go b/cmd/zed/dev/vector/project/command.go index ec9021455b..935c6c1e01 100644 --- a/cmd/zed/dev/vector/project/command.go +++ b/cmd/zed/dev/vector/project/command.go @@ -64,15 +64,14 @@ func (c *Command) Run(args []string) error { for _, dotted := range args[1:] { paths = append(paths, field.Dotted(dotted)) } - zctx := zed.NewContext() local := storage.NewLocalEngine() cache := vcache.NewCache(local) - object, err := cache.Fetch(ctx, zctx, uri, ksuid.Nil) + object, err := cache.Fetch(ctx, uri, ksuid.Nil) if err != nil { return err } defer object.Close() - projection := vam.NewProjection(zctx, object, paths) + projection := vam.NewProjection(zed.NewContext(), object, paths) writer, err := c.outputFlags.Open(ctx, local) if err != nil { return err diff --git a/cmd/zed/dev/vector/query/command.go b/cmd/zed/dev/vector/query/command.go index b5d4513cc3..1fd42e7ff4 100644 --- a/cmd/zed/dev/vector/query/command.go +++ b/cmd/zed/dev/vector/query/command.go @@ -62,15 +62,14 @@ func (c *Command) Run(args []string) error { if err != nil { return err } - zctx := zed.NewContext() local := storage.NewLocalEngine() cache := vcache.NewCache(local) - object, err := cache.Fetch(ctx, zctx, uri, ksuid.Nil) + object, err := cache.Fetch(ctx, uri, ksuid.Nil) if err != nil { return err } defer object.Close() - rctx := runtime.NewContext(ctx, zctx) + rctx := runtime.NewContext(ctx, zed.NewContext()) puller, err := compiler.VectorCompile(rctx, text, object) if err != nil { return err diff --git a/cmd/zed/dev/vng/command.go b/cmd/zed/dev/vng/command.go index 308266747b..7b74350d3d 100644 --- a/cmd/zed/dev/vng/command.go +++ b/cmd/zed/dev/vng/command.go @@ -79,7 +79,6 @@ type reader struct { reader *bufio.Reader meta *zngio.Reader marshaler *zson.MarshalZNGContext - arena *zed.Arena dataSize int } @@ -91,7 +90,6 @@ func newReader(r io.Reader) *reader { zctx: zctx, reader: bufio.NewReader(r), marshaler: zson.NewZNGMarshalerWithContext(zctx), - arena: zed.NewArena(), } } @@ -107,8 +105,7 @@ func (r *reader) Read() (*zed.Value, error) { } r.meta = zngio.NewReader(r.zctx, io.LimitReader(r.reader, int64(hdr.MetaSize))) r.dataSize = int(hdr.DataSize) - r.arena.Reset() - val, err := r.marshaler.Marshal(r.arena, hdr) + val, err := r.marshaler.Marshal(hdr) return val.Ptr(), err } val, err := r.meta.Read() diff --git a/cmd/zed/internal/lakemanage/scan.go b/cmd/zed/internal/lakemanage/scan.go index 93d59e6702..5a75c7c231 100644 --- a/cmd/zed/internal/lakemanage/scan.go +++ b/cmd/zed/internal/lakemanage/scan.go @@ -70,7 +70,6 @@ type objectIterator struct { reader zio.Reader puller zbuf.Puller unmarshaler *zson.UnmarshalZNGContext - arena *zed.Arena } func newObjectIterator(ctx context.Context, lake api.Interface, head *lakeparse.Commitish) (*objectIterator, error) { @@ -79,12 +78,10 @@ func newObjectIterator(ctx context.Context, lake api.Interface, head *lakeparse. if err != nil { return nil, err } - arena := zed.NewArena() return &objectIterator{ reader: zbuf.PullerReader(q), puller: q, - unmarshaler: zson.NewZNGUnmarshaler().SetContext(zed.NewContext(), arena), - arena: arena, + unmarshaler: zson.NewZNGUnmarshaler(), }, nil } @@ -99,7 +96,6 @@ func (r *objectIterator) next() (*object, error) { if err := r.unmarshaler.Unmarshal(*val, &o.Object); err != nil { return nil, err } - o.Object.Arena = r.arena if err := r.unmarshaler.Unmarshal(*val, &o); err != nil { return nil, err } diff --git a/compiler/kernel/bufferfilter.go b/compiler/kernel/bufferfilter.go index b86b539663..29fd347ffd 100644 --- a/compiler/kernel/bufferfilter.go +++ b/compiler/kernel/bufferfilter.go @@ -16,9 +16,7 @@ import ( func CompileBufferFilter(zctx *zed.Context, e dag.Expr) (*expr.BufferFilter, error) { switch e := e.(type) { case *dag.BinaryExpr: - arena := zed.NewArena() - defer arena.Unref() - literal, err := isFieldEqualOrIn(zctx, arena, e) + literal, err := isFieldEqualOrIn(zctx, e) if err != nil { return nil, err } @@ -55,9 +53,7 @@ func CompileBufferFilter(zctx *zed.Context, e dag.Expr) (*expr.BufferFilter, err } return nil, nil case *dag.Search: - arena := zed.NewArena() - defer arena.Unref() - literal, err := zson.ParseValue(zctx, arena, e.Value) + literal, err := zson.ParseValue(zctx, e.Value) if err != nil { return nil, err } @@ -84,10 +80,10 @@ func CompileBufferFilter(zctx *zed.Context, e dag.Expr) (*expr.BufferFilter, err } } -func isFieldEqualOrIn(zctx *zed.Context, arena *zed.Arena, e *dag.BinaryExpr) (*zed.Value, error) { +func isFieldEqualOrIn(zctx *zed.Context, e *dag.BinaryExpr) (*zed.Value, error) { if _, ok := e.LHS.(*dag.This); ok && e.Op == "==" { if literal, ok := e.RHS.(*dag.Literal); ok { - val, err := zson.ParseValue(zctx, arena, literal.Value) + val, err := zson.ParseValue(zctx, literal.Value) if err != nil { return nil, err } @@ -95,7 +91,7 @@ func isFieldEqualOrIn(zctx *zed.Context, arena *zed.Arena, e *dag.BinaryExpr) (* } } else if _, ok := e.RHS.(*dag.This); ok && e.Op == "in" { if literal, ok := e.LHS.(*dag.Literal); ok { - val, err := zson.ParseValue(zctx, arena, literal.Value) + val, err := zson.ParseValue(zctx, literal.Value) if err != nil { return nil, err } diff --git a/compiler/kernel/expr.go b/compiler/kernel/expr.go index e9570dd28b..65fcc44e8f 100644 --- a/compiler/kernel/expr.go +++ b/compiler/kernel/expr.go @@ -52,7 +52,7 @@ func (b *Builder) compileExpr(e dag.Expr) (expr.Evaluator, error) { } switch e := e.(type) { case *dag.Literal: - val, err := zson.ParseValue(b.zctx(), b.arena, e.Value) + val, err := zson.ParseValue(b.zctx(), e.Value) if err != nil { return nil, err } @@ -194,7 +194,7 @@ func (b *Builder) compileConstCompare(e *dag.BinaryExpr) (expr.Evaluator, error) } func (b *Builder) compileSearch(search *dag.Search) (expr.Evaluator, error) { - val, err := zson.ParseValue(b.zctx(), b.arena, search.Value) + val, err := zson.ParseValue(b.zctx(), search.Value) if err != nil { return nil, err } diff --git a/compiler/kernel/filter.go b/compiler/kernel/filter.go index 675dcd006c..51b50a630c 100644 --- a/compiler/kernel/filter.go +++ b/compiler/kernel/filter.go @@ -1,7 +1,6 @@ package kernel import ( - "github.com/brimdata/zed" "github.com/brimdata/zed/compiler/ast/dag" "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zbuf" @@ -18,12 +17,7 @@ func (f *Filter) AsEvaluator() (expr.Evaluator, error) { if f == nil { return nil, nil } - arena := zed.NewArena() - e, err := f.builder.clone(arena).compileExpr(f.pushdown) - if err != nil { - return nil, err - } - return &arenaEvaluator{e, arena}, nil + return f.builder.compileExpr(f.pushdown) } func (f *Filter) AsBufferFilter() (*expr.BufferFilter, error) { @@ -41,12 +35,11 @@ func (f *DeleteFilter) AsEvaluator() (expr.Evaluator, error) { if f == nil { return nil, nil } - arena := zed.NewArena() // For a DeleteFilter Evaluator the pushdown gets wrapped in a unary ! // expression so we get all values that don't match. We also add a missing // call so if the expression results in an error("missing") the value is // kept. - e, err := f.builder.clone(arena).compileExpr(&dag.BinaryExpr{ + return f.builder.compileExpr(&dag.BinaryExpr{ Kind: "BinaryExpr", Op: "or", LHS: &dag.UnaryExpr{ @@ -60,17 +53,8 @@ func (f *DeleteFilter) AsEvaluator() (expr.Evaluator, error) { Args: []dag.Expr{f.pushdown}, }, }) - if err != nil { - return nil, err - } - return &arenaEvaluator{e, arena}, nil } func (f *DeleteFilter) AsBufferFilter() (*expr.BufferFilter, error) { return nil, nil } - -type arenaEvaluator struct { - expr.Evaluator - arena *zed.Arena -} diff --git a/compiler/kernel/op.go b/compiler/kernel/op.go index 1fb348d4d7..0df7e5eb04 100644 --- a/compiler/kernel/op.go +++ b/compiler/kernel/op.go @@ -56,7 +56,6 @@ type Builder struct { source *data.Source readers []zio.Reader progress *zbuf.Progress - arena *zed.Arena // For zed.Values created during compilation. channels map[string][]zbuf.Puller deletes *sync.Map udfs map[string]dag.Expr @@ -65,8 +64,6 @@ type Builder struct { } func NewBuilder(rctx *runtime.Context, source *data.Source) *Builder { - arena := zed.NewArena() - rctx.KeepAlive(arena) return &Builder{ rctx: rctx, mctx: zed.NewContext(), @@ -77,19 +74,12 @@ func NewBuilder(rctx *runtime.Context, source *data.Source) *Builder { RecordsRead: 0, RecordsMatched: 0, }, - arena: arena, channels: make(map[string][]zbuf.Puller), udfs: make(map[string]dag.Expr), compiledUDFs: make(map[string]*expr.UDF), } } -func (b *Builder) clone(arena *zed.Arena) *Builder { - bb := *b - bb.arena = arena - return &bb -} - // Build builds a flowgraph for seq. If seq contains a dag.DefaultSource, it // will read from readers. func (b *Builder) Build(seq dag.Seq, readers ...zio.Reader) (map[string]zbuf.Puller, error) { @@ -294,7 +284,7 @@ func (b *Builder) compileLeaf(o dag.Op, parent zbuf.Puller) (zbuf.Puller, error) var pruner expr.Evaluator if v.Tap && v.KeyPruner != nil { var err error - pruner, err = b.compileExpr(v.KeyPruner) + pruner, err = compileExpr(v.KeyPruner) if err != nil { return nil, err } @@ -331,7 +321,7 @@ func (b *Builder) compileLeaf(o dag.Op, parent zbuf.Puller) (zbuf.Puller, error) } var pruner expr.Evaluator if v.KeyPruner != nil { - pruner, err = b.compileExpr(v.KeyPruner) + pruner, err = compileExpr(v.KeyPruner) if err != nil { return nil, err } @@ -346,7 +336,7 @@ func (b *Builder) compileLeaf(o dag.Op, parent zbuf.Puller) (zbuf.Puller, error) } var pruner expr.Evaluator if v.KeyPruner != nil { - pruner, err = b.compileExpr(v.KeyPruner) + pruner, err = compileExpr(v.KeyPruner) if err != nil { return nil, err } @@ -359,7 +349,7 @@ func (b *Builder) compileLeaf(o dag.Op, parent zbuf.Puller) (zbuf.Puller, error) } var pruner expr.Evaluator if v.KeyPruner != nil { - pruner, err = b.compileExpr(v.KeyPruner) + pruner, err = compileExpr(v.KeyPruner) if err != nil { return nil, err } @@ -733,22 +723,25 @@ func (b *Builder) evalAtCompileTime(in dag.Expr) (val zed.Value, err error) { if err != nil { return zed.Null, err } - missing := b.zctx().Missing(b.arena) // Catch panic as the runtime will panic if there is a // reference to a var not in scope, a field access null this, etc. defer func() { if recover() != nil { - val = missing + val = b.zctx().Missing() } }() - return e.Eval(expr.NewContext(b.arena), missing), nil + return e.Eval(expr.NewContext(), b.zctx().Missing()), nil +} + +func compileExpr(in dag.Expr) (expr.Evaluator, error) { + b := NewBuilder(runtime.NewContext(context.Background(), zed.NewContext()), nil) + return b.compileExpr(in) } -func EvalAtCompileTime(zctx *zed.Context, arena *zed.Arena, in dag.Expr) (zed.Value, error) { +func EvalAtCompileTime(zctx *zed.Context, in dag.Expr) (val zed.Value, err error) { // We pass in a nil adaptor, which causes a panic for anything adaptor // related, which is not currently allowed in an expression sub-query. b := NewBuilder(runtime.NewContext(context.Background(), zctx), nil) - b.arena = arena return b.evalAtCompileTime(in) } diff --git a/compiler/kernel/vexpr.go b/compiler/kernel/vexpr.go index 7aee830e96..002d6bbcf3 100644 --- a/compiler/kernel/vexpr.go +++ b/compiler/kernel/vexpr.go @@ -17,11 +17,11 @@ func (b *Builder) compileVamExpr(e dag.Expr) (vamexpr.Evaluator, error) { } switch e := e.(type) { case *dag.Literal: - val, err := zson.ParseValue(b.zctx(), b.arena, e.Value) + val, err := zson.ParseValue(b.zctx(), e.Value) if err != nil { return nil, err } - return vamexpr.NewLiteral(b.arena, val), nil + return vamexpr.NewLiteral(val), nil //case *dag.Var: // return vamexpr.NewVar(e.Slot), nil //case *dag.Search: diff --git a/compiler/semantic/analyzer.go b/compiler/semantic/analyzer.go index 6270555117..6f9dd5635f 100644 --- a/compiler/semantic/analyzer.go +++ b/compiler/semantic/analyzer.go @@ -51,7 +51,6 @@ type analyzer struct { source *data.Source scope *Scope zctx *zed.Context - arena *zed.Arena } func newAnalyzer(ctx context.Context, source *data.Source, head *lakeparse.Commitish) *analyzer { @@ -62,7 +61,6 @@ func newAnalyzer(ctx context.Context, source *data.Source, head *lakeparse.Commi source: source, scope: NewScope(nil), zctx: zed.NewContext(), - arena: zed.NewArena(), } } diff --git a/compiler/semantic/expr.go b/compiler/semantic/expr.go index 39443f674a..b50b647cb9 100644 --- a/compiler/semantic/expr.go +++ b/compiler/semantic/expr.go @@ -35,7 +35,7 @@ func (a *analyzer) semExpr(e ast.Expr) dag.Expr { case *ast.Grep: return a.semGrep(e) case *astzed.Primitive: - val, err := zson.ParsePrimitive(a.arena, e.Type, e.Text) + val, err := zson.ParsePrimitive(e.Type, e.Text) if err != nil { a.error(e, err) return badExpr() @@ -50,7 +50,7 @@ func (a *analyzer) semExpr(e ast.Expr) dag.Expr { var val string switch t := e.Value.(type) { case *astzed.Primitive: - v, err := zson.ParsePrimitive(a.arena, t.Type, t.Text) + v, err := zson.ParsePrimitive(t.Type, t.Text) if err != nil { a.error(e, err) return badExpr() @@ -290,7 +290,7 @@ func (a *analyzer) semGrep(grep *ast.Grep) dag.Expr { s.Expr = e return s } - if s, ok := a.isStringConst(p); ok { + if s, ok := isStringConst(a.zctx, p); ok { return &dag.Search{ Kind: "Search", Text: s, @@ -360,7 +360,7 @@ func (a *analyzer) semBinary(e *ast.BinaryExpr) dag.Expr { func (a *analyzer) isIndexOfThis(lhs, rhs dag.Expr) *dag.This { if this, ok := lhs.(*dag.This); ok { - if s, ok := a.isStringConst(rhs); ok { + if s, ok := isStringConst(a.zctx, rhs); ok { this.Path = append(this.Path, s) return this } @@ -368,8 +368,8 @@ func (a *analyzer) isIndexOfThis(lhs, rhs dag.Expr) *dag.This { return nil } -func (a *analyzer) isStringConst(e dag.Expr) (field string, ok bool) { - val, err := kernel.EvalAtCompileTime(a.zctx, a.arena, e) +func isStringConst(zctx *zed.Context, e dag.Expr) (field string, ok bool) { + val, err := kernel.EvalAtCompileTime(zctx, e) if err == nil && !val.IsError() && zed.TypeUnder(val.Type()) == zed.TypeString { return string(val.Bytes()), true } diff --git a/compiler/semantic/op.go b/compiler/semantic/op.go index e9526e483b..4161b33f47 100644 --- a/compiler/semantic/op.go +++ b/compiler/semantic/op.go @@ -114,11 +114,11 @@ func (a *analyzer) semSource(source ast.Source) []dag.Op { var headers map[string][]string if s.Headers != nil { expr := a.semExpr(s.Headers) - val, err := kernel.EvalAtCompileTime(a.zctx, a.arena, expr) + val, err := kernel.EvalAtCompileTime(a.zctx, expr) if err != nil { a.error(s.Headers, err) } else { - headers, err = unmarshalHeaders(a.arena, val) + headers, err = unmarshalHeaders(val) if err != nil { a.error(s.Headers, err) } @@ -167,7 +167,7 @@ func (a *analyzer) semSource(source ast.Source) []dag.Op { } } -func unmarshalHeaders(arena *zed.Arena, val zed.Value) (map[string][]string, error) { +func unmarshalHeaders(val zed.Value) (map[string][]string, error) { if !zed.IsRecordType(val.Type()) { return nil, errors.New("headers value must be a record") } @@ -176,7 +176,7 @@ func unmarshalHeaders(arena *zed.Arena, val zed.Value) (map[string][]string, err if inner := zed.InnerType(f.Type); inner == nil || inner.ID() != zed.IDString { return nil, errors.New("headers field value must be an array or set of strings") } - fieldVal := val.DerefByColumn(arena, i) + fieldVal := val.DerefByColumn(i) if fieldVal == nil { continue } @@ -260,7 +260,7 @@ func (a *analyzer) maybeStringConst(name string) (string, error) { if !ok { return "", fmt.Errorf("%s: string value required", name) } - val := zson.MustParseValue(a.zctx, a.arena, l.Value) + val := zson.MustParseValue(a.zctx, l.Value) if val.Type().ID() != zed.IDString { return "", fmt.Errorf("%s: string value required", name) } @@ -526,7 +526,7 @@ func (a *analyzer) semOp(o ast.Op, seq dag.Seq) dag.Seq { if o.Count != nil { expr := a.semExpr(o.Count) var err error - if val, err = kernel.EvalAtCompileTime(a.zctx, a.arena, expr); err != nil { + if val, err = kernel.EvalAtCompileTime(a.zctx, expr); err != nil { a.error(o.Count, err) return append(seq, badOp()) } @@ -547,7 +547,7 @@ func (a *analyzer) semOp(o ast.Op, seq dag.Seq) dag.Seq { if o.Count != nil { expr := a.semExpr(o.Count) var err error - if val, err = kernel.EvalAtCompileTime(a.zctx, a.arena, expr); err != nil { + if val, err = kernel.EvalAtCompileTime(a.zctx, expr); err != nil { a.error(o.Count, err) return append(seq, badOp()) } @@ -586,7 +586,7 @@ func (a *analyzer) semOp(o ast.Op, seq dag.Seq) dag.Seq { limit := 1 if o.Limit != nil { l := a.semExpr(o.Limit) - val, err := kernel.EvalAtCompileTime(a.zctx, a.arena, l) + val, err := kernel.EvalAtCompileTime(a.zctx, l) if err != nil { a.error(o.Limit, err) return append(seq, badOp()) @@ -868,7 +868,7 @@ func (a *analyzer) semDecls(decls []ast.Decl) ([]dag.Def, []*dag.Func) { func (a *analyzer) semConstDecl(c *ast.ConstDecl) dag.Def { e := a.semExpr(c.Expr) - if err := a.scope.DefineConst(a.zctx, a.arena, c.Name, e); err != nil { + if err := a.scope.DefineConst(a.zctx, c.Name, e); err != nil { a.error(c, err) } return dag.Def{ @@ -887,7 +887,7 @@ func (a *analyzer) semTypeDecl(d *ast.TypeDecl) dag.Def { Kind: "Literal", Value: fmt.Sprintf("<%s=%s>", zson.QuotedName(d.Name.Name), typ), } - if err := a.scope.DefineConst(a.zctx, a.arena, d.Name, e); err != nil { + if err := a.scope.DefineConst(a.zctx, d.Name, e); err != nil { a.error(d.Name, err) } return dag.Def{Name: d.Name.Name, Expr: e} @@ -1113,7 +1113,7 @@ func (a *analyzer) maybeConvertUserOp(call *ast.Call) dag.Seq { e := a.semExpr(arg) // Transform non-path arguments into literals. if _, ok := e.(*dag.This); !ok { - val, err := kernel.EvalAtCompileTime(a.zctx, a.arena, e) + val, err := kernel.EvalAtCompileTime(a.zctx, e) if err != nil { a.error(arg, err) exprs[i] = badExpr() diff --git a/compiler/semantic/scope.go b/compiler/semantic/scope.go index 696808ec18..8fb55cb96c 100644 --- a/compiler/semantic/scope.go +++ b/compiler/semantic/scope.go @@ -51,8 +51,8 @@ func (s *Scope) DefineAs(name *ast.ID, e any) error { return nil } -func (s *Scope) DefineConst(zctx *zed.Context, arena *zed.Arena, name *ast.ID, def dag.Expr) error { - val, err := kernel.EvalAtCompileTime(zctx, arena, def) +func (s *Scope) DefineConst(zctx *zed.Context, name *ast.ID, def dag.Expr) error { + val, err := kernel.EvalAtCompileTime(zctx, def) if err != nil { return err } diff --git a/complex_test.go b/complex_test.go index 656f84522e..be9381a891 100644 --- a/complex_test.go +++ b/complex_test.go @@ -15,12 +15,10 @@ import ( func TestRecordAccessNamed(t *testing.T) { const input = `{foo:"hello" (=zfile),bar:true (=zbool)} (=0)` - arena := zed.NewArena() - defer arena.Unref() - rec := zson.MustParseValue(zed.NewContext(), arena, input) - s := rec.Deref(arena, "foo").AsString() + rec := zson.MustParseValue(zed.NewContext(), input) + s := rec.Deref("foo").AsString() assert.Equal(t, s, "hello") - b := rec.Deref(arena, "bar").AsBool() + b := rec.Deref("bar").AsBool() assert.Equal(t, b, true) } @@ -31,8 +29,6 @@ func TestNonRecordDeref(t *testing.T) { null [1,2,3] |[1,2,3]|` - arena := zed.NewArena() - defer arena.Unref() reader := zsonio.NewReader(zed.NewContext(), strings.NewReader(input)) for { val, err := reader.Read() @@ -40,7 +36,7 @@ null break } require.NoError(t, err) - v := val.Deref(arena, "foo") + v := val.Deref("foo") require.Nil(t, v) } } diff --git a/context.go b/context.go index 11f738d2f0..abaceca7c7 100644 --- a/context.go +++ b/context.go @@ -311,12 +311,12 @@ func (c *Context) enterWithLock(tv zcode.Bytes, typ Type) { c.byID = append(c.byID, typ) } -func (c *Context) LookupTypeValue(arena *Arena, typ Type) Value { +func (c *Context) LookupTypeValue(typ Type) Value { c.mu.Lock() bytes, ok := c.toValue[typ] c.mu.Unlock() if ok { - return arena.New(TypeType, bytes) + return NewValue(TypeType, bytes) } // In general, this shouldn't happen except for a foreign // type that wasn't initially created in this context. @@ -326,9 +326,9 @@ func (c *Context) LookupTypeValue(arena *Arena, typ Type) Value { typ, err := c.LookupByValue(tv) if err != nil { // This shouldn't happen. - return c.Missing(arena) + return c.Missing() } - return c.LookupTypeValue(arena, typ) + return c.LookupTypeValue(typ) } func (c *Context) DecodeTypeValue(tv zcode.Bytes) (Type, zcode.Bytes) { @@ -491,20 +491,22 @@ func DecodeLength(tv zcode.Bytes) (int, zcode.Bytes) { return int(namelen), tv[n:] } -func (c *Context) Missing(arena *Arena) Value { - return arena.New(c.StringTypeError(), Missing) +func (c *Context) Missing() Value { + return NewValue(c.StringTypeError(), Missing) } -func (c *Context) Quiet(arena *Arena) Value { - return arena.New(c.StringTypeError(), Quiet) +func (c *Context) Quiet() Value { + return NewValue(c.StringTypeError(), Quiet) } -func (c *Context) NewErrorf(arena *Arena, format string, args ...interface{}) Value { - return arena.New(c.StringTypeError(), fmt.Appendf(nil, format, args...)) +// batch/allocator should handle these? + +func (c *Context) NewErrorf(format string, args ...interface{}) Value { + return NewValue(c.StringTypeError(), fmt.Appendf(nil, format, args...)) } -func (c *Context) NewError(arena *Arena, err error) Value { - return arena.New(c.StringTypeError(), []byte(err.Error())) +func (c *Context) NewError(err error) Value { + return NewValue(c.StringTypeError(), []byte(err.Error())) } func (c *Context) StringTypeError() *TypeError { @@ -514,7 +516,7 @@ func (c *Context) StringTypeError() *TypeError { return c.LookupTypeError(TypeString) } -func (c *Context) WrapError(arena *Arena, msg string, val Value) Value { +func (c *Context) WrapError(msg string, val Value) Value { recType := c.MustLookupTypeRecord([]Field{ {"message", TypeString}, {"on", val.Type()}, @@ -523,5 +525,5 @@ func (c *Context) WrapError(arena *Arena, msg string, val Value) Value { var b zcode.Builder b.Append(EncodeString(msg)) b.Append(val.Bytes()) - return arena.New(errType, b.Bytes()) + return NewValue(errType, b.Bytes()) } diff --git a/context_test.go b/context_test.go index f3f4c11b50..0ce8305265 100644 --- a/context_test.go +++ b/context_test.go @@ -44,9 +44,7 @@ func TestContextTranslateTypeNameConflictUnion(t *testing.T) { // top level typedef in TranslateType so foo in the value below had // two of the same union type instead of the two it should have had. zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - val := zson.MustParseValue(zctx, arena, `[{x:{y:63}}(=foo),{x:{abcdef:{x:{y:127}}(foo)}}(=foo)]`) + val := zson.MustParseValue(zctx, `[{x:{y:63}}(=foo),{x:{abcdef:{x:{y:127}}(foo)}}(=foo)]`) foreign := zed.NewContext() twin, err := foreign.TranslateType(val.Type()) require.NoError(t, err) diff --git a/docs/libraries/go.md b/docs/libraries/go.md index bf57d722d8..2ba0c4bb0c 100644 --- a/docs/libraries/go.md +++ b/docs/libraries/go.md @@ -55,7 +55,6 @@ import ( func main() { zctx := zed.NewContext() reader := zsonio.NewReader(zctx, os.Stdin) - arena := zed.NewArena() for { val, err := reader.Read() if err != nil { @@ -64,9 +63,9 @@ func main() { if val == nil { return } - s := val.Deref(arena, "s") + s := val.Deref("s") if s == nil { - s = zctx.Missing(arena).Ptr() + s = zctx.Missing().Ptr() } fmt.Println(zson.String(s)) } @@ -142,7 +141,6 @@ func main() { defer q.Pull(true) reader := zbuf.PullerReader(q) zctx := zed.NewContext() - arena := zed.NewArena() for { val, err := reader.Read() if err != nil { @@ -151,9 +149,9 @@ func main() { if val == nil { return } - s := val.Deref(arena, "s") + s := val.Deref("s") if s == nil { - s = zctx.Missing(arena).Ptr() + s = zctx.Missing().Ptr() } fmt.Println(zson.String(s)) } diff --git a/fuzz/fuzz.go b/fuzz/fuzz.go index 8bc9702b2c..81314fc7ba 100644 --- a/fuzz/fuzz.go +++ b/fuzz/fuzz.go @@ -31,7 +31,7 @@ import ( "github.com/x448/float16" ) -func ReadZNG(bs []byte) (zbuf.Batch, error) { +func ReadZNG(bs []byte) ([]zed.Value, error) { bytesReader := bytes.NewReader(bs) context := zed.NewContext() reader := zngio.NewReader(context, bytesReader) @@ -41,10 +41,10 @@ func ReadZNG(bs []byte) (zbuf.Batch, error) { if err != nil { return nil, err } - return &a, nil + return a.Values(), nil } -func ReadVNG(bs []byte, demandOut demand.Demand) (zbuf.Batch, error) { +func ReadVNG(bs []byte, demandOut demand.Demand) ([]zed.Value, error) { bytesReader := bytes.NewReader(bs) context := zed.NewContext() reader, err := vngio.NewReader(context, bytesReader, demandOut) @@ -56,29 +56,29 @@ func ReadVNG(bs []byte, demandOut demand.Demand) (zbuf.Batch, error) { if err != nil { return nil, err } - return &a, nil + return a.Values(), nil } -func WriteZNG(t testing.TB, batch zbuf.Batch, buf *bytes.Buffer) { +func WriteZNG(t testing.TB, valuesIn []zed.Value, buf *bytes.Buffer) { writer := zngio.NewWriter(zio.NopCloser(buf)) - require.NoError(t, zbuf.WriteBatch(writer, batch)) + require.NoError(t, zio.Copy(writer, zbuf.NewArray(valuesIn))) require.NoError(t, writer.Close()) } -func WriteVNG(t testing.TB, batch zbuf.Batch, buf *bytes.Buffer) { +func WriteVNG(t testing.TB, valuesIn []zed.Value, buf *bytes.Buffer) { writer := vngio.NewWriter(zio.NopCloser(buf)) - require.NoError(t, zbuf.WriteBatch(writer, batch)) + require.NoError(t, zio.Copy(writer, zbuf.NewArray(valuesIn))) require.NoError(t, writer.Close()) } -func RunQueryZNG(t testing.TB, buf *bytes.Buffer, querySource string) zbuf.Batch { +func RunQueryZNG(t testing.TB, buf *bytes.Buffer, querySource string) []zed.Value { zctx := zed.NewContext() readers := []zio.Reader{zngio.NewReader(zctx, buf)} defer zio.CloseReaders(readers) return RunQuery(t, zctx, readers, querySource, func(_ demand.Demand) {}) } -func RunQueryVNG(t testing.TB, buf *bytes.Buffer, querySource string) zbuf.Batch { +func RunQueryVNG(t testing.TB, buf *bytes.Buffer, querySource string) []zed.Value { zctx := zed.NewContext() reader, err := vngio.NewReader(zctx, bytes.NewReader(buf.Bytes()), demand.All()) require.NoError(t, err) @@ -87,7 +87,7 @@ func RunQueryVNG(t testing.TB, buf *bytes.Buffer, querySource string) zbuf.Batch return RunQuery(t, zctx, readers, querySource, func(_ demand.Demand) {}) } -func RunQuery(t testing.TB, zctx *zed.Context, readers []zio.Reader, querySource string, useDemand func(demandIn demand.Demand)) zbuf.Batch { +func RunQuery(t testing.TB, zctx *zed.Context, readers []zio.Reader, querySource string, useDemand func(demandIn demand.Demand)) []zed.Value { ctx := context.Background() ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -119,7 +119,7 @@ func RunQuery(t testing.TB, zctx *zed.Context, readers []zio.Reader, querySource } // Run query - var a zbuf.Array + var valuesOut []zed.Value for { batch, err := query.Pull(false) require.NoError(t, err) @@ -127,11 +127,12 @@ func RunQuery(t testing.TB, zctx *zed.Context, readers []zio.Reader, querySource break } for _, value := range batch.Values() { - a.Write(value) + valuesOut = append(valuesOut, value.Copy()) } batch.Unref() } - return &a + + return valuesOut } func CompareValues(t testing.TB, valuesExpected []zed.Value, valuesActual []zed.Value) { @@ -158,18 +159,16 @@ func CompareValues(t testing.TB, valuesExpected []zed.Value, valuesActual []zed. } } -func GenValues(b *bytes.Reader, context *zed.Context, types []zed.Type) zbuf.Batch { - arena := zed.NewArena() - defer arena.Unref() +func GenValues(b *bytes.Reader, context *zed.Context, types []zed.Type) []zed.Value { var values []zed.Value var builder zcode.Builder for GenByte(b) != 0 { typ := types[int(GenByte(b))%len(types)] builder.Reset() GenValue(b, context, typ, &builder) - values = append(values, arena.New(typ, builder.Bytes().Body())) + values = append(values, zed.NewValue(typ, builder.Bytes().Body())) } - return zbuf.NewArray(arena, values) + return values } func GenValue(b *bytes.Reader, context *zed.Context, typ zed.Type, builder *zcode.Builder) { diff --git a/lake/branch.go b/lake/branch.go index 078789d72f..53c0d223e1 100644 --- a/lake/branch.go +++ b/lake/branch.go @@ -65,9 +65,7 @@ func (b *Branch) Load(ctx context.Context, zctx *zed.Context, r zio.Reader, auth if message == "" { message = loadMessage(objects) } - arena := zed.NewArena() - defer arena.Unref() - appMeta, err := loadMeta(zctx, arena, meta) + appMeta, err := loadMeta(zctx, meta) if err != nil { return ksuid.Nil, err } @@ -95,13 +93,13 @@ func loadMessage(objects []data.Object) string { return b.String() } -func loadMeta(zctx *zed.Context, arena *zed.Arena, meta string) (zed.Value, error) { +func loadMeta(zctx *zed.Context, meta string) (zed.Value, error) { if meta == "" { return zed.Null, nil } - val, err := zson.ParseValue(zctx, arena, meta) + val, err := zson.ParseValue(zed.NewContext(), meta) if err != nil { - return zed.Null, fmt.Errorf("%w %q: %s", ErrInvalidCommitMeta, meta, err) + return zctx.Missing(), fmt.Errorf("%w %q: %s", ErrInvalidCommitMeta, meta, err) } return val, nil } @@ -132,9 +130,7 @@ func (b *Branch) Delete(ctx context.Context, ids []ksuid.KSUID, author, message func (b *Branch) DeleteWhere(ctx context.Context, c runtime.Compiler, program ast.Seq, author, message, meta string) (ksuid.KSUID, error) { zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - appMeta, err := loadMeta(zctx, arena, meta) + appMeta, err := loadMeta(zctx, meta) if err != nil { return ksuid.Nil, err } @@ -243,9 +239,8 @@ func (b *Branch) CommitCompact(ctx context.Context, src, rollup []*data.Object, if len(rollup) < 1 { return ksuid.Nil, errors.New("compact: one or more rollup objects required") } - arena := zed.NewArena() - defer arena.Unref() - appMeta, err := loadMeta(zed.NewContext(), arena, meta) + zctx := zed.NewContext() + appMeta, err := loadMeta(zctx, meta) if err != nil { return ksuid.Nil, err } diff --git a/lake/commits/actions.go b/lake/commits/actions.go index 73ce8de77b..7070cd8927 100644 --- a/lake/commits/actions.go +++ b/lake/commits/actions.go @@ -22,15 +22,6 @@ var ActionTypes = []interface{}{ Commit{}, } -func setActionArena(action Action, arena *zed.Arena) { - switch action := action.(type) { - case *Add: - action.Object.Arena = arena - case *Commit: - action.arena = arena - } -} - type Add struct { Commit ksuid.KSUID `zed:"commit"` Object data.Object `zed:"object"` @@ -61,7 +52,6 @@ type Commit struct { Date nano.Ts `zed:"date"` Message string `zed:"message"` Meta zed.Value `zed:"meta"` - arena *zed.Arena } func (c *Commit) CommitID() ksuid.KSUID { diff --git a/lake/commits/object.go b/lake/commits/object.go index 5403a2e337..91ea8d5ff3 100644 --- a/lake/commits/object.go +++ b/lake/commits/object.go @@ -110,10 +110,9 @@ func (o Object) Serialize() ([]byte, error) { return b, nil } -func DecodeObject(zctx *zed.Context, r io.Reader) (*Object, error) { - arena := zed.NewArena() +func DecodeObject(r io.Reader) (*Object, error) { o := &Object{} - reader := zngbytes.NewDeserializer(zctx, arena, r, ActionTypes) + reader := zngbytes.NewDeserializer(r, ActionTypes) defer reader.Close() for { entry, err := reader.Read() @@ -127,7 +126,6 @@ func DecodeObject(zctx *zed.Context, r io.Reader) (*Object, error) { if !ok { return nil, badObject(entry) } - setActionArena(action, arena) o.append(action) } // Fill in the commit and parent IDs from the first record, diff --git a/lake/commits/reader.go b/lake/commits/reader.go index d05ce6a4e1..a2cde5ff6a 100644 --- a/lake/commits/reader.go +++ b/lake/commits/reader.go @@ -12,7 +12,6 @@ import ( type LogReader struct { ctx context.Context marshaler *zson.MarshalZNGContext - arena *zed.Arena store *Store cursor ksuid.KSUID stop ksuid.KSUID @@ -26,7 +25,6 @@ func newLogReader(ctx context.Context, zctx *zed.Context, store *Store, leaf, st return &LogReader{ ctx: ctx, marshaler: m, - arena: zed.NewArena(), store: store, cursor: leaf, stop: stop, @@ -46,7 +44,6 @@ func (r *LogReader) Read() (*zed.Value, error) { next = ksuid.Nil } r.cursor = next - r.arena.Reset() - val, err := r.marshaler.Marshal(r.arena, commitObject) + val, err := r.marshaler.Marshal(commitObject) return &val, err } diff --git a/lake/commits/snapshot.go b/lake/commits/snapshot.go index 46e8549610..221f904f89 100644 --- a/lake/commits/snapshot.go +++ b/lake/commits/snapshot.go @@ -5,7 +5,6 @@ import ( "fmt" "io" - "github.com/brimdata/zed" "github.com/brimdata/zed/lake/data" "github.com/brimdata/zed/order" "github.com/brimdata/zed/runtime/sam/expr/extent" @@ -157,10 +156,9 @@ func (s *Snapshot) serialize() ([]byte, error) { return zs.Bytes(), nil } -func decodeSnapshot(zctx *zed.Context, r io.Reader) (*Snapshot, error) { - arena := zed.NewArena() +func decodeSnapshot(r io.Reader) (*Snapshot, error) { s := NewSnapshot() - zd := zngbytes.NewDeserializer(zctx, arena, r, ActionTypes) + zd := zngbytes.NewDeserializer(r, ActionTypes) defer zd.Close() for { entry, err := zd.Read() @@ -174,7 +172,6 @@ func decodeSnapshot(zctx *zed.Context, r io.Reader) (*Snapshot, error) { if !ok { return nil, fmt.Errorf("internal error: corrupt snapshot contains unknown entry type %T", entry) } - setActionArena(action, arena) if err := PlayAction(s, action); err != nil { return nil, err } diff --git a/lake/commits/store.go b/lake/commits/store.go index fa68555bea..3b96d0525c 100644 --- a/lake/commits/store.go +++ b/lake/commits/store.go @@ -34,7 +34,6 @@ type Store struct { cache *lru.ARCCache[ksuid.KSUID, *Object] paths *lru.ARCCache[ksuid.KSUID, []ksuid.KSUID] snapshots *lru.ARCCache[ksuid.KSUID, *Snapshot] - zctx *zed.Context } func OpenStore(engine storage.Engine, logger *zap.Logger, path *storage.URI) (*Store, error) { @@ -57,7 +56,6 @@ func OpenStore(engine storage.Engine, logger *zap.Logger, path *storage.URI) (*S cache: cache, paths: paths, snapshots: snapshots, - zctx: zed.NewContext(), }, nil } @@ -69,7 +67,7 @@ func (s *Store) Get(ctx context.Context, commit ksuid.KSUID) (*Object, error) { if err != nil { return nil, err } - o, err := DecodeObject(s.zctx, r) + o, err := DecodeObject(r) if err == ErrBadCommitObject { err = fmt.Errorf("system error: %s: %w", s.pathOf(commit), ErrBadCommitObject) } @@ -168,7 +166,7 @@ func (s *Store) getSnapshot(ctx context.Context, commit ksuid.KSUID) (*Snapshot, return nil, err } defer r.Close() - return decodeSnapshot(s.zctx, r) + return decodeSnapshot(r) } func (s *Store) putSnapshot(ctx context.Context, commit ksuid.KSUID, snap *Snapshot) error { @@ -230,8 +228,7 @@ func (s *Store) GetBytes(ctx context.Context, commit ksuid.KSUID) ([]byte, *Comm if err != nil { return nil, nil, err } - arena := zed.NewArena() - reader := zngbytes.NewDeserializer(s.zctx, arena, bytes.NewReader(b), ActionTypes) + reader := zngbytes.NewDeserializer(bytes.NewReader(b), ActionTypes) defer reader.Close() entry, err := reader.Read() if err != nil { @@ -241,7 +238,6 @@ func (s *Store) GetBytes(ctx context.Context, commit ksuid.KSUID) ([]byte, *Comm if !ok { return nil, nil, fmt.Errorf("system error: first record of commit object is not a commit action: %s", s.pathOf(commit)) } - setActionArena(entry.(Action), arena) return b, first, nil } diff --git a/lake/data/object.go b/lake/data/object.go index 566d7767bb..6cb19ea157 100644 --- a/lake/data/object.go +++ b/lake/data/object.go @@ -81,7 +81,6 @@ type Object struct { Max zed.Value `zed:"max"` Count uint64 `zed:"count"` Size int64 `zed:"size"` - Arena *zed.Arena `zed:"-"` } func (o Object) IsZero() bool { diff --git a/lake/data/seekindex.go b/lake/data/seekindex.go index d4fc9fe171..e257e0c86c 100644 --- a/lake/data/seekindex.go +++ b/lake/data/seekindex.go @@ -25,14 +25,10 @@ func LookupSeekRange(ctx context.Context, engine storage.Engine, path *storage.U } defer r.Close() var ranges seekindex.Ranges - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() unmarshaler := zson.NewZNGUnmarshaler() - unmarshaler.SetContext(zctx, arena) - reader := zngio.NewReader(zctx, r) + reader := zngio.NewReader(zed.NewContext(), r) defer reader.Close() - ectx := expr.NewContext(arena) + ectx := expr.NewContext() for { val, err := reader.Read() if val == nil || err != nil { @@ -52,25 +48,21 @@ func LookupSeekRange(ctx context.Context, engine storage.Engine, path *storage.U func RangeFromBitVector(ctx context.Context, engine storage.Engine, path *storage.URI, o *Object, b *vector.Bool) ([]seekindex.Range, error) { - arena := zed.NewArena() - defer arena.Unref() - index, err := readSeekIndex(ctx, arena, engine, path, o) + index, err := readSeekIndex(ctx, engine, path, o) if err != nil { return nil, err } return index.Filter(b), nil } -func readSeekIndex(ctx context.Context, arena *zed.Arena, engine storage.Engine, path *storage.URI, o *Object) (seekindex.Index, error) { +func readSeekIndex(ctx context.Context, engine storage.Engine, path *storage.URI, o *Object) (seekindex.Index, error) { r, err := engine.Get(ctx, o.SeekIndexURI(path)) if err != nil { return nil, err } defer r.Close() - zctx := zed.NewContext() - zr := zngio.NewReader(zctx, r) + zr := zngio.NewReader(zed.NewContext(), r) u := zson.NewZNGUnmarshaler() - u.SetContext(zctx, arena) var index seekindex.Index for { val, err := zr.Read() diff --git a/lake/data/writer.go b/lake/data/writer.go index c566d98ccd..cf6ee8ec00 100644 --- a/lake/data/writer.go +++ b/lake/data/writer.go @@ -25,9 +25,6 @@ type Writer struct { seekIndexTrigger int first bool seekMin *zed.Value - keyArena *zed.Arena - seekMinArena *zed.Arena - maxArena *zed.Arena } // NewWriter returns a writer for writing the data of a zng-row storage object as @@ -42,14 +39,11 @@ func (o *Object) NewWriter(ctx context.Context, engine storage.Engine, path *sto } counter := &writeCounter{bufwriter.New(out), 0} w := &Writer{ - object: o, - byteCounter: counter, - writer: zngio.NewWriter(counter), - sortKey: sortKey, - first: true, - keyArena: zed.NewArena(), - seekMinArena: zed.NewArena(), - maxArena: zed.NewArena(), + object: o, + byteCounter: counter, + writer: zngio.NewWriter(counter), + sortKey: sortKey, + first: true, } if seekIndexStride == 0 { seekIndexStride = DefaultSeekStride @@ -64,8 +58,7 @@ func (o *Object) NewWriter(ctx context.Context, engine storage.Engine, path *sto } func (w *Writer) Write(val zed.Value) error { - w.keyArena.Reset() - key := val.DerefPath(w.keyArena, w.sortKey.Key).MissingAsNull() + key := val.DerefPath(w.sortKey.Key).MissingAsNull() return w.WriteWithKey(key, val) } @@ -74,8 +67,7 @@ func (w *Writer) WriteWithKey(key, val zed.Value) error { if err := w.writer.Write(val); err != nil { return err } - w.maxArena.Reset() - w.object.Max = key.Copy(w.maxArena) + w.object.Max.CopyFrom(key) return w.writeIndex(key) } @@ -83,12 +75,10 @@ func (w *Writer) writeIndex(key zed.Value) error { w.seekIndexTrigger += len(key.Bytes()) if w.first { w.first = false - w.object.Arena = zed.NewArena() - w.object.Min = key.Copy(w.object.Arena) + w.object.Min.CopyFrom(key) } if w.seekMin == nil { - w.seekMinArena, w.keyArena = w.keyArena, w.seekMinArena - w.seekMin = &key + w.seekMin = key.Copy().Ptr() } if w.seekIndexTrigger < w.seekIndexStride { return nil @@ -135,7 +125,6 @@ func (w *Writer) Close(ctx context.Context) error { } w.object.Count = w.count w.object.Size = w.writer.Position() - w.object.Max = w.object.Max.Copy(w.object.Arena) if w.sortKey.Order == order.Desc { w.object.Min, w.object.Max = w.object.Max, w.object.Min } diff --git a/lake/data/writer_test.go b/lake/data/writer_test.go index 69d1062562..0504ec09c9 100644 --- a/lake/data/writer_test.go +++ b/lake/data/writer_test.go @@ -2,7 +2,6 @@ package data_test import ( "context" - "runtime" "testing" "github.com/brimdata/zed" @@ -25,19 +24,16 @@ func TestDataReaderWriterVector(t *testing.T) { w, err := object.NewWriter(ctx, engine, tmp, order.NewSortKey(order.Asc, field.Path{"a"}), 1000) require.NoError(t, err) zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - require.NoError(t, w.Write(zson.MustParseValue(zctx, arena, "{a:1,b:4}"))) - require.NoError(t, w.Write(zson.MustParseValue(zctx, arena, "{a:2,b:5}"))) - require.NoError(t, w.Write(zson.MustParseValue(zctx, arena, "{a:3,b:6}"))) + require.NoError(t, w.Write(zson.MustParseValue(zctx, "{a:1,b:4}"))) + require.NoError(t, w.Write(zson.MustParseValue(zctx, "{a:2,b:5}"))) + require.NoError(t, w.Write(zson.MustParseValue(zctx, "{a:3,b:6}"))) require.NoError(t, w.Close(ctx)) require.NoError(t, data.CreateVector(ctx, engine, tmp, object.ID)) // Read back the VNG file and make sure it's the same. get, err := engine.Get(ctx, object.VectorURI(tmp)) require.NoError(t, err) - reader, err := vngio.NewReader(zctx, get, demand.All()) + reader, err := vngio.NewReader(zed.NewContext(), get, demand.All()) require.NoError(t, err) - defer runtime.KeepAlive(reader) v, err := reader.Read() require.NoError(t, err) assert.Equal(t, zson.String(v), "{a:1,b:4}") diff --git a/lake/journal/store.go b/lake/journal/store.go index 6750a0b018..68da6f2bc4 100644 --- a/lake/journal/store.go +++ b/lake/journal/store.go @@ -189,11 +189,8 @@ func (s *Store) putSnapshot(ctx context.Context, at ID, table map[string]Entry) } marshaler := zson.NewZNGMarshaler() marshaler.Decorate(zson.StylePackage) - arena := zed.NewArena() - defer arena.Unref() for _, entry := range table { - arena.Reset() - val, err := marshaler.Marshal(arena, entry) + val, err := marshaler.Marshal(entry) if err != nil { return err } diff --git a/lake/pool.go b/lake/pool.go index d2eb24744a..76417473fc 100644 --- a/lake/pool.go +++ b/lake/pool.go @@ -156,15 +156,15 @@ func (p *Pool) ResolveRevision(ctx context.Context, revision string) (ksuid.KSUI return id, nil } -func (p *Pool) BatchifyBranches(ctx context.Context, zctx *zed.Context, arena *zed.Arena, recs []zed.Value, m *zson.MarshalZNGContext, f expr.Evaluator) ([]zed.Value, error) { +func (p *Pool) BatchifyBranches(ctx context.Context, zctx *zed.Context, recs []zed.Value, m *zson.MarshalZNGContext, f expr.Evaluator) ([]zed.Value, error) { branches, err := p.ListBranches(ctx) if err != nil { return nil, err } - ectx := expr.NewContext(arena) + ectx := expr.NewContext() for _, branchRef := range branches { meta := BranchMeta{p.Config, branchRef} - rec, err := m.Marshal(arena, &meta) + rec, err := m.Marshal(&meta) if err != nil { return nil, err } @@ -188,7 +188,7 @@ type BranchTip struct { Commit ksuid.KSUID } -func (p *Pool) BatchifyBranchTips(ctx context.Context, zctx *zed.Context, arena *zed.Arena, f expr.Evaluator) ([]zed.Value, error) { +func (p *Pool) BatchifyBranchTips(ctx context.Context, zctx *zed.Context, f expr.Evaluator) ([]zed.Value, error) { branches, err := p.ListBranches(ctx) if err != nil { return nil, err @@ -196,9 +196,9 @@ func (p *Pool) BatchifyBranchTips(ctx context.Context, zctx *zed.Context, arena m := zson.NewZNGMarshalerWithContext(zctx) m.Decorate(zson.StylePackage) recs := make([]zed.Value, 0, len(branches)) - ectx := expr.NewContext(arena) + ectx := expr.NewContext() for _, branchRef := range branches { - rec, err := m.Marshal(arena, &BranchTip{branchRef.Name, branchRef.Commit}) + rec, err := m.Marshal(&BranchTip{branchRef.Name, branchRef.Commit}) if err != nil { return nil, err } diff --git a/lake/root.go b/lake/root.go index 903cac1f1c..89e8ef04b3 100644 --- a/lake/root.go +++ b/lake/root.go @@ -162,10 +162,6 @@ func (r *Root) readLakeMagic(ctx context.Context) error { if err != nil { return err } - var magic LakeMagic - if err := zson.UnmarshalZNG(nil, nil, *val, &magic); err != nil { - return fmt.Errorf("corrupt lake version file: %w", err) - } last, err := zr.Read() if err != nil { return err @@ -173,6 +169,10 @@ func (r *Root) readLakeMagic(ctx context.Context) error { if last != nil { return fmt.Errorf("corrupt lake version file: more than one Zed value at %s", zson.String(last)) } + var magic LakeMagic + if err := zson.UnmarshalZNG(*val, &magic); err != nil { + return fmt.Errorf("corrupt lake version file: %w", err) + } if magic.Magic != LakeMagicString { return fmt.Errorf("corrupt lake version file: magic %q should be %q", magic.Magic, LakeMagicString) } @@ -182,17 +182,17 @@ func (r *Root) readLakeMagic(ctx context.Context) error { return nil } -func (r *Root) BatchifyPools(ctx context.Context, zctx *zed.Context, arena *zed.Arena, f expr.Evaluator) ([]zed.Value, error) { +func (r *Root) BatchifyPools(ctx context.Context, zctx *zed.Context, f expr.Evaluator) ([]zed.Value, error) { m := zson.NewZNGMarshalerWithContext(zctx) m.Decorate(zson.StylePackage) pools, err := r.ListPools(ctx) if err != nil { return nil, err } - ectx := expr.NewContext(arena) + ectx := expr.NewContext() var vals []zed.Value for k := range pools { - rec, err := m.Marshal(arena, &pools[k]) + rec, err := m.Marshal(&pools[k]) if err != nil { return nil, err } @@ -203,7 +203,7 @@ func (r *Root) BatchifyPools(ctx context.Context, zctx *zed.Context, arena *zed. return vals, nil } -func (r *Root) BatchifyBranches(ctx context.Context, zctx *zed.Context, arena *zed.Arena, f expr.Evaluator) ([]zed.Value, error) { +func (r *Root) BatchifyBranches(ctx context.Context, zctx *zed.Context, f expr.Evaluator) ([]zed.Value, error) { m := zson.NewZNGMarshalerWithContext(zctx) m.Decorate(zson.StylePackage) poolRefs, err := r.ListPools(ctx) @@ -221,7 +221,7 @@ func (r *Root) BatchifyBranches(ctx context.Context, zctx *zed.Context, arena *z } return nil, err } - vals, err = pool.BatchifyBranches(ctx, zctx, arena, vals, m, f) + vals, err = pool.BatchifyBranches(ctx, zctx, vals, m, f) if err != nil { return nil, err } diff --git a/lake/seekindex/writer.go b/lake/seekindex/writer.go index 98efcf2166..f2c3b0ed51 100644 --- a/lake/seekindex/writer.go +++ b/lake/seekindex/writer.go @@ -8,7 +8,6 @@ import ( type Writer struct { marshal *zson.MarshalZNGContext - arena *zed.Arena writer zio.WriteCloser offset uint64 valoff uint64 @@ -17,14 +16,12 @@ type Writer struct { func NewWriter(w zio.WriteCloser) *Writer { return &Writer{ marshal: zson.NewZNGMarshaler(), - arena: zed.NewArena(), writer: w, } } func (w *Writer) Write(min, max zed.Value, valoff uint64, offset uint64) error { - w.arena.Reset() - val, err := w.marshal.Marshal(w.arena, &Entry{ + val, err := w.marshal.Marshal(&Entry{ Min: min, Max: max, ValOff: w.valoff, diff --git a/lake/writer.go b/lake/writer.go index 14ea535b13..f8d20c033d 100644 --- a/lake/writer.go +++ b/lake/writer.go @@ -23,12 +23,15 @@ type Writer struct { objects []data.Object inputSorted bool ctx context.Context + zctx *zed.Context errgroup *errgroup.Group - array *zbuf.Array - // This channel implements a simple double buffering model so the - // cloud-object writer can run in parallel with the reader filling the - // records buffer. - buffer chan *zbuf.Array + vals []zed.Value + // XXX this is a simple double buffering model so the cloud-object + // writer can run in parallel with the reader filling the records + // buffer. This can be later extended to pass a big bytes buffer + // back and forth where the bytes buffer holds all of the record + // data efficiently in one big backing store. + buffer chan []zed.Value comparator *expr.Comparator memBuffered int64 stats ImportStats @@ -48,13 +51,13 @@ type Writer struct { // to do useful things like paritioning given the context is a rollup. func NewWriter(ctx context.Context, zctx *zed.Context, pool *Pool) (*Writer, error) { g, ctx := errgroup.WithContext(ctx) - ch := make(chan *zbuf.Array, 1) - ch <- zbuf.NewArray(nil, nil) + ch := make(chan []zed.Value, 1) + ch <- nil return &Writer{ pool: pool, ctx: ctx, + zctx: zctx, errgroup: g, - array: zbuf.NewArray(nil, nil), buffer: ch, comparator: ImportComparator(zctx, pool), }, nil @@ -76,7 +79,11 @@ func (w *Writer) Write(rec zed.Value) error { } return w.ctx.Err() } - w.array.Write(rec) + // XXX This call leads to a ton of one-off allocations that burden the GC + // and slow down import. We should instead copy the raw record bytes into a + // recycled buffer and keep around an array of ts + byte-slice structs for + // sorting. + w.vals = append(w.vals, rec.Copy()) w.memBuffered += int64(len(rec.Bytes())) //XXX change name LogSizeThreshold // XXX the previous logic estimated the object size with divide by 2...?! @@ -87,21 +94,20 @@ func (w *Writer) Write(rec zed.Value) error { } func (w *Writer) flipBuffers() { - oldArray, ok := <-w.buffer + oldvals, ok := <-w.buffer if !ok { return } - oldArray.Reset() - array := w.array - w.array = oldArray + recs := w.vals + w.vals = oldvals[:0] w.memBuffered = 0 w.errgroup.Go(func() error { - err := w.writeObject(w.newObject(), array.Values()) + err := w.writeObject(w.newObject(), recs) if err != nil { close(w.buffer) return err } - w.buffer <- array + w.buffer <- recs return err }) } @@ -109,7 +115,7 @@ func (w *Writer) flipBuffers() { func (w *Writer) Close() error { // Send the last write (Note: we could reorder things so we do the // record sort in this thread while waiting for the write to complete.) - if len(w.array.Values()) > 0 { + if len(w.vals) > 0 { w.flipBuffers() } // Wait for any pending write to finish. @@ -119,7 +125,7 @@ func (w *Writer) Close() error { func (w *Writer) writeObject(object *data.Object, recs []zed.Value) error { var zr zio.Reader if w.inputSorted { - zr = zbuf.NewArray(nil, recs) + zr = zbuf.NewArray(recs) } else { done := make(chan struct{}) go func() { @@ -165,8 +171,6 @@ type SortedWriter struct { vectorEnabled bool vectorWriter *data.VectorWriter objects []*data.Object - keyArena *zed.Arena - lastKeyArena *zed.Arena } func NewSortedWriter(ctx context.Context, zctx *zed.Context, pool *Pool, vectorEnabled bool) *SortedWriter { @@ -176,14 +180,11 @@ func NewSortedWriter(ctx context.Context, zctx *zed.Context, pool *Pool, vectorE sortKey: pool.SortKeys.Primary(), pool: pool, vectorEnabled: vectorEnabled, - keyArena: zed.NewArena(), - lastKeyArena: zed.NewArena(), } } func (w *SortedWriter) Write(val zed.Value) error { - w.keyArena.Reset() - key := val.DerefPath(w.keyArena, w.sortKey.Key).MissingAsNull() + key := val.DerefPath(w.sortKey.Key).MissingAsNull() again: if w.writer == nil { if err := w.newWriter(); err != nil { @@ -210,8 +211,7 @@ again: return err } } - w.keyArena, w.lastKeyArena = w.lastKeyArena, w.keyArena - w.lastKey = key + w.lastKey.CopyFrom(key) return nil } diff --git a/runtime/context.go b/runtime/context.go index 3aed144d93..e0506a6b18 100644 --- a/runtime/context.go +++ b/runtime/context.go @@ -16,7 +16,6 @@ type Context struct { WaitGroup sync.WaitGroup Zctx *zed.Context cancel context.CancelFunc - keepalive []*zed.Arena } func NewContext(ctx context.Context, zctx *zed.Context) *Context { @@ -38,8 +37,3 @@ func (c *Context) Cancel() { c.cancel() c.WaitGroup.Wait() } - -// KeepAlive ensures that arena is not freed before the context. -func (c *Context) KeepAlive(arena *zed.Arena) { - c.keepalive = append(c.keepalive, arena) -} diff --git a/runtime/sam/expr/agg.go b/runtime/sam/expr/agg.go index 2e875852aa..3ad812d111 100644 --- a/runtime/sam/expr/agg.go +++ b/runtime/sam/expr/agg.go @@ -66,7 +66,7 @@ func (s *AggregatorExpr) Eval(ectx Context, val zed.Value) zed.Value { s.fn = s.agg.NewFunction() } s.agg.Apply(s.zctx, ectx, s.fn, val) - return s.fn.Result(s.zctx, ectx.Arena()) + return s.fn.Result(s.zctx) } func (s *AggregatorExpr) Reset() { diff --git a/runtime/sam/expr/agg/agg.go b/runtime/sam/expr/agg/agg.go index 2fca365709..b0bf7b9d29 100644 --- a/runtime/sam/expr/agg/agg.go +++ b/runtime/sam/expr/agg/agg.go @@ -19,8 +19,8 @@ type Pattern func() Function type Function interface { Consume(zed.Value) ConsumeAsPartial(zed.Value) - Result(*zed.Context, *zed.Arena) zed.Value - ResultAsPartial(*zed.Context, *zed.Arena) zed.Value + Result(*zed.Context) zed.Value + ResultAsPartial(*zed.Context) zed.Value } func NewPattern(op string, hasarg bool) (Pattern, error) { @@ -35,11 +35,11 @@ func NewPattern(op string, hasarg bool) (Pattern, error) { } case "any": pattern = func() Function { - return newAny() + return &Any{} } case "avg": pattern = func() Function { - return newAvg() + return &Avg{} } case "dcount": pattern = func() Function { @@ -71,7 +71,7 @@ func NewPattern(op string, hasarg bool) (Pattern, error) { } case "collect": pattern = func() Function { - return newCollect() + return &Collect{} } case "and": pattern = func() Function { diff --git a/runtime/sam/expr/agg/any.go b/runtime/sam/expr/agg/any.go index ac269ff9d1..72388718e5 100644 --- a/runtime/sam/expr/agg/any.go +++ b/runtime/sam/expr/agg/any.go @@ -4,36 +4,34 @@ import ( "github.com/brimdata/zed" ) -type Any struct { - arena *zed.Arena - val *zed.Value -} +type Any zed.Value var _ Function = (*Any)(nil) -func newAny() *Any { - return &Any{arena: zed.NewArena()} +func NewAny() *Any { + a := (Any)(zed.Null) + return &a } func (a *Any) Consume(val zed.Value) { // Copy any value from the input while favoring any-typed non-null values // over null values. - if a.val == nil || a.val.IsNull() && !val.IsNull() { - a.val = val.Copy(a.arena).Ptr() + if (*zed.Value)(a).Type() == nil || (*zed.Value)(a).IsNull() && !val.IsNull() { + *a = Any(val.Copy()) } } -func (a *Any) Result(*zed.Context, *zed.Arena) zed.Value { - if a.val == nil { +func (a *Any) Result(*zed.Context) zed.Value { + if (*zed.Value)(a).Type() == nil { return zed.Null } - return *a.val + return *(*zed.Value)(a) } func (a *Any) ConsumeAsPartial(v zed.Value) { a.Consume(v) } -func (a *Any) ResultAsPartial(*zed.Context, *zed.Arena) zed.Value { - return a.Result(nil, nil) +func (a *Any) ResultAsPartial(*zed.Context) zed.Value { + return a.Result(nil) } diff --git a/runtime/sam/expr/agg/avg.go b/runtime/sam/expr/agg/avg.go index f6948706ab..bb425e4587 100644 --- a/runtime/sam/expr/agg/avg.go +++ b/runtime/sam/expr/agg/avg.go @@ -11,17 +11,12 @@ import ( ) type Avg struct { - arena *zed.Arena sum float64 count uint64 } var _ Function = (*Avg)(nil) -func newAvg() *Avg { - return &Avg{arena: zed.NewArena()} -} - func (a *Avg) Consume(val zed.Value) { if val.IsNull() { return @@ -32,7 +27,7 @@ func (a *Avg) Consume(val zed.Value) { } } -func (a *Avg) Result(*zed.Context, *zed.Arena) zed.Value { +func (a *Avg) Result(*zed.Context) zed.Value { if a.count > 0 { return zed.NewFloat64(a.sum / float64(a.count)) } @@ -45,15 +40,14 @@ const ( ) func (a *Avg) ConsumeAsPartial(partial zed.Value) { - a.arena.Reset() - sumVal := partial.Deref(a.arena, sumName) + sumVal := partial.Deref(sumName) if sumVal == nil { panic(errors.New("avg: partial sum is missing")) } if sumVal.Type() != zed.TypeFloat64 { panic(fmt.Errorf("avg: partial sum has bad type: %s", zson.FormatValue(*sumVal))) } - countVal := partial.Deref(a.arena, countName) + countVal := partial.Deref(countName) if countVal == nil { panic("avg: partial count is missing") } @@ -64,7 +58,7 @@ func (a *Avg) ConsumeAsPartial(partial zed.Value) { a.count += countVal.Uint() } -func (a *Avg) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { +func (a *Avg) ResultAsPartial(zctx *zed.Context) zed.Value { var zv zcode.Bytes zv = zed.NewFloat64(a.sum).Encode(zv) zv = zed.NewUint64(a.count).Encode(zv) @@ -72,5 +66,5 @@ func (a *Avg) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { zed.NewField(sumName, zed.TypeFloat64), zed.NewField(countName, zed.TypeUint64), }) - return arena.New(typ, zv) + return zed.NewValue(typ, zv) } diff --git a/runtime/sam/expr/agg/collect.go b/runtime/sam/expr/agg/collect.go index 96f8cb05b1..1877eb048b 100644 --- a/runtime/sam/expr/agg/collect.go +++ b/runtime/sam/expr/agg/collect.go @@ -9,17 +9,12 @@ import ( ) type Collect struct { - arena *zed.Arena values []zed.Value size int } var _ Function = (*Collect)(nil) -func newCollect() *Collect { - return &Collect{arena: zed.NewArena()} -} - func (c *Collect) Consume(val zed.Value) { if !val.IsNull() { c.update(val) @@ -27,7 +22,7 @@ func (c *Collect) Consume(val zed.Value) { } func (c *Collect) update(val zed.Value) { - c.values = append(c.values, val.Under(c.arena).Copy(c.arena)) + c.values = append(c.values, val.Under().Copy()) c.size += len(val.Bytes()) for c.size > MaxValueSize { // XXX See issue #1813. For now we silently discard entries @@ -38,7 +33,7 @@ func (c *Collect) update(val zed.Value) { } } -func (c *Collect) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { +func (c *Collect) Result(zctx *zed.Context) zed.Value { if len(c.values) == 0 { // no values found return zed.Null @@ -54,7 +49,7 @@ func (c *Collect) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { b.Append(val.Bytes()) } } - return arena.New(zctx.LookupTypeArray(inner), b.Bytes()) + return zed.NewValue(zctx.LookupTypeArray(inner), b.Bytes()) } func innerType(zctx *zed.Context, vals []zed.Value) zed.Type { @@ -80,10 +75,10 @@ func (c *Collect) ConsumeAsPartial(val zed.Value) { } typ := arrayType.Type for it := val.Iter(); !it.Done(); { - c.update(c.arena.New(typ, it.Next())) + c.update(zed.NewValue(typ, it.Next())) } } -func (c *Collect) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { - return c.Result(zctx, arena) +func (c *Collect) ResultAsPartial(zctx *zed.Context) zed.Value { + return c.Result(zctx) } diff --git a/runtime/sam/expr/agg/collectmap.go b/runtime/sam/expr/agg/collectmap.go index ee7ff91421..3d5a890ab9 100644 --- a/runtime/sam/expr/agg/collectmap.go +++ b/runtime/sam/expr/agg/collectmap.go @@ -8,16 +8,12 @@ import ( ) type CollectMap struct { - arena *zed.Arena entries map[string]mapEntry scratch []byte } func newCollectMap() *CollectMap { - return &CollectMap{ - arena: zed.NewArena(), - entries: make(map[string]mapEntry), - } + return &CollectMap{entries: make(map[string]mapEntry)} } var _ Function = (*Collect)(nil) @@ -39,8 +35,8 @@ func (c *CollectMap) Consume(val zed.Value) { it := zcode.Iter(slices.Clone(val.Bytes())) for !it.Done() { keyTagAndBody := it.NextTagAndBody() - key := valueUnder(c.arena, mtyp.KeyType, keyTagAndBody.Body()) - val := valueUnder(c.arena, mtyp.ValType, it.Next()) + key := valueUnder(mtyp.KeyType, keyTagAndBody.Body()) + val := valueUnder(mtyp.ValType, it.Next()) c.scratch = zed.AppendTypeValue(c.scratch[:0], key.Type()) c.scratch = append(c.scratch, keyTagAndBody...) // This will squash existing values which is what we want. @@ -52,7 +48,7 @@ func (c *CollectMap) ConsumeAsPartial(val zed.Value) { c.Consume(val) } -func (c *CollectMap) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { +func (c *CollectMap) Result(zctx *zed.Context) zed.Value { if len(c.entries) == 0 { return zed.Null } @@ -73,11 +69,11 @@ func (c *CollectMap) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { } typ := zctx.LookupTypeMap(ktyp, vtyp) b := zed.NormalizeMap(builder.Bytes()) - return arena.New(typ, b) + return zed.NewValue(typ, b) } -func (c *CollectMap) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { - return c.Result(zctx, arena) +func (c *CollectMap) ResultAsPartial(zctx *zed.Context) zed.Value { + return c.Result(zctx) } func appendMapVal(b *zcode.Builder, typ zed.Type, val zed.Value, uniq int) { @@ -98,10 +94,10 @@ func unionOf(zctx *zed.Context, types []zed.Type) (zed.Type, int) { } // valueUnder is like zed.(*Value).Under but it preserves non-union named types. -func valueUnder(arena *zed.Arena, typ zed.Type, b zcode.Bytes) zed.Value { - val := arena.New(typ, b) +func valueUnder(typ zed.Type, b zcode.Bytes) zed.Value { + val := zed.NewValue(typ, b) if _, ok := zed.TypeUnder(typ).(*zed.TypeUnion); !ok { return val } - return val.Under(arena) + return val.Under() } diff --git a/runtime/sam/expr/agg/count.go b/runtime/sam/expr/agg/count.go index b5cfb497f9..4e43709857 100644 --- a/runtime/sam/expr/agg/count.go +++ b/runtime/sam/expr/agg/count.go @@ -12,7 +12,7 @@ func (c *Count) Consume(zed.Value) { *c++ } -func (c Count) Result(*zed.Context, *zed.Arena) zed.Value { +func (c Count) Result(*zed.Context) zed.Value { return zed.NewUint64(uint64(c)) } @@ -23,6 +23,6 @@ func (c *Count) ConsumeAsPartial(partial zed.Value) { *c += Count(partial.Uint()) } -func (c Count) ResultAsPartial(*zed.Context, *zed.Arena) zed.Value { - return c.Result(nil, nil) +func (c Count) ResultAsPartial(*zed.Context) zed.Value { + return c.Result(nil) } diff --git a/runtime/sam/expr/agg/dcount.go b/runtime/sam/expr/agg/dcount.go index 24a74d8709..64da6b0964 100644 --- a/runtime/sam/expr/agg/dcount.go +++ b/runtime/sam/expr/agg/dcount.go @@ -33,7 +33,7 @@ func (d *DCount) Consume(val zed.Value) { d.sketch.Insert(d.scratch) } -func (d *DCount) Result(*zed.Context, *zed.Arena) zed.Value { +func (d *DCount) Result(*zed.Context) zed.Value { return zed.NewUint64(d.sketch.Estimate()) } @@ -48,10 +48,10 @@ func (d *DCount) ConsumeAsPartial(partial zed.Value) { d.sketch.Merge(&s) } -func (d *DCount) ResultAsPartial(_ *zed.Context, arena *zed.Arena) zed.Value { +func (d *DCount) ResultAsPartial(zctx *zed.Context) zed.Value { b, err := d.sketch.MarshalBinary() if err != nil { panic(fmt.Errorf("dcount: marshaling partial: %w", err)) } - return arena.NewBytes(b) + return zed.NewBytes(b) } diff --git a/runtime/sam/expr/agg/fuse.go b/runtime/sam/expr/agg/fuse.go index eba7315f1b..2073d409b8 100644 --- a/runtime/sam/expr/agg/fuse.go +++ b/runtime/sam/expr/agg/fuse.go @@ -7,17 +7,15 @@ import ( ) type fuse struct { - shapes map[zed.Type]int - partials []zed.Value - partialsArena *zed.Arena + shapes map[zed.Type]int + partials []zed.Value } var _ Function = (*fuse)(nil) func newFuse() *fuse { return &fuse{ - shapes: make(map[zed.Type]int), - partialsArena: zed.NewArena(), + shapes: make(map[zed.Type]int), } } @@ -27,7 +25,7 @@ func (f *fuse) Consume(val zed.Value) { } } -func (f *fuse) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { +func (f *fuse) Result(zctx *zed.Context) zed.Value { if len(f.shapes)+len(f.partials) == 0 { return zed.NullType } @@ -46,16 +44,16 @@ func (f *fuse) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { for _, typ := range shapes { schema.Mixin(typ) } - return zctx.LookupTypeValue(arena, schema.Type()) + return zctx.LookupTypeValue(schema.Type()) } func (f *fuse) ConsumeAsPartial(partial zed.Value) { if partial.Type() != zed.TypeType { panic("fuse: partial not a type value") } - f.partials = append(f.partials, partial.Copy(f.partialsArena)) + f.partials = append(f.partials, partial.Copy()) } -func (f *fuse) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { - return f.Result(zctx, arena) +func (f *fuse) ResultAsPartial(zctx *zed.Context) zed.Value { + return f.Result(zctx) } diff --git a/runtime/sam/expr/agg/logical.go b/runtime/sam/expr/agg/logical.go index d032bc106a..a1538eaade 100644 --- a/runtime/sam/expr/agg/logical.go +++ b/runtime/sam/expr/agg/logical.go @@ -21,7 +21,7 @@ func (a *And) Consume(val zed.Value) { *a.val = *a.val && val.Bool() } -func (a *And) Result(*zed.Context, *zed.Arena) zed.Value { +func (a *And) Result(*zed.Context) zed.Value { if a.val == nil { return zed.NullBool } @@ -35,8 +35,8 @@ func (a *And) ConsumeAsPartial(val zed.Value) { a.Consume(val) } -func (a *And) ResultAsPartial(*zed.Context, *zed.Arena) zed.Value { - return a.Result(nil, nil) +func (a *And) ResultAsPartial(*zed.Context) zed.Value { + return a.Result(nil) } type Or struct { @@ -56,7 +56,7 @@ func (o *Or) Consume(val zed.Value) { *o.val = *o.val || val.Bool() } -func (o *Or) Result(*zed.Context, *zed.Arena) zed.Value { +func (o *Or) Result(*zed.Context) zed.Value { if o.val == nil { return zed.NullBool } @@ -70,6 +70,6 @@ func (o *Or) ConsumeAsPartial(val zed.Value) { o.Consume(val) } -func (o *Or) ResultAsPartial(*zed.Context, *zed.Arena) zed.Value { - return o.Result(nil, nil) +func (o *Or) ResultAsPartial(*zed.Context) zed.Value { + return o.Result(nil) } diff --git a/runtime/sam/expr/agg/math.go b/runtime/sam/expr/agg/math.go index 1616ef7e45..5f779cb895 100644 --- a/runtime/sam/expr/agg/math.go +++ b/runtime/sam/expr/agg/math.go @@ -28,12 +28,12 @@ func newMathReducer(f *anymath.Function) *mathReducer { return &mathReducer{function: f} } -func (m *mathReducer) Result(_ *zed.Context, arena *zed.Arena) zed.Value { +func (m *mathReducer) Result(zctx *zed.Context) zed.Value { if !m.hasval { if m.math == nil { return zed.Null } - return arena.New(m.math.typ(), nil) + return zed.NewValue(m.math.typ(), nil) } return m.math.result() } @@ -46,9 +46,7 @@ func (m *mathReducer) consumeVal(val zed.Value) { var id int if m.math != nil { var err error - arena := zed.NewArena() - defer arena.Unref() - id, err = coerce.Promote(arena.New(m.math.typ(), nil), val) + id, err = coerce.Promote(zed.NewValue(m.math.typ(), nil), val) if err != nil { // Skip invalid values. return @@ -84,8 +82,8 @@ func (m *mathReducer) consumeVal(val zed.Value) { m.math.consume(val) } -func (m *mathReducer) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { - return m.Result(zctx, arena) +func (m *mathReducer) ResultAsPartial(*zed.Context) zed.Value { + return m.Result(nil) } func (m *mathReducer) ConsumeAsPartial(val zed.Value) { diff --git a/runtime/sam/expr/agg/union.go b/runtime/sam/expr/agg/union.go index 971775e4da..c50f0aa304 100644 --- a/runtime/sam/expr/agg/union.go +++ b/runtime/sam/expr/agg/union.go @@ -56,7 +56,7 @@ func (u *Union) deleteOne() { } } -func (u *Union) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { +func (u *Union) Result(zctx *zed.Context) zed.Value { if len(u.types) == 0 { return zed.Null } @@ -80,7 +80,7 @@ func (u *Union) Result(zctx *zed.Context, arena *zed.Arena) zed.Value { b.Append([]byte(v)) } } - return arena.New(zctx.LookupTypeSet(inner), zed.NormalizeSet(b.Bytes())) + return zed.NewValue(zctx.LookupTypeSet(inner), zed.NormalizeSet(b.Bytes())) } func (u *Union) ConsumeAsPartial(val zed.Value) { @@ -101,6 +101,6 @@ func (u *Union) ConsumeAsPartial(val zed.Value) { } } -func (u *Union) ResultAsPartial(zctx *zed.Context, arena *zed.Arena) zed.Value { - return u.Result(zctx, arena) +func (u *Union) ResultAsPartial(zctx *zed.Context) zed.Value { + return u.Result(zctx) } diff --git a/runtime/sam/expr/boolean.go b/runtime/sam/expr/boolean.go index ca2b046fc2..8f4f485c20 100644 --- a/runtime/sam/expr/boolean.go +++ b/runtime/sam/expr/boolean.go @@ -225,11 +225,9 @@ func CompareNull(op string) (Boolean, error) { // Given a predicate for comparing individual elements, produce a new // predicate that implements the "in" comparison. func Contains(compare Boolean) Boolean { - arena := zed.NewArena() return func(val zed.Value) bool { return errMatch == val.Walk(func(typ zed.Type, body zcode.Bytes) error { - arena.Reset() - if compare(arena.New(typ, body)) { + if compare(zed.NewValue(typ, body)) { return errMatch } return nil diff --git a/runtime/sam/expr/cast.go b/runtime/sam/expr/cast.go index 277b60d981..42e942fd89 100644 --- a/runtime/sam/expr/cast.go +++ b/runtime/sam/expr/cast.go @@ -69,7 +69,7 @@ type casterIntN struct { func (c *casterIntN) Eval(ectx Context, val zed.Value) zed.Value { v, ok := coerce.ToInt(val) if !ok || (c.min != 0 && (v < c.min || v > c.max)) { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to "+zson.FormatType(c.typ), val) + return c.zctx.WrapError("cannot cast to "+zson.FormatType(c.typ), val) } return zed.NewInt(c.typ, v) } @@ -83,7 +83,7 @@ type casterUintN struct { func (c *casterUintN) Eval(ectx Context, val zed.Value) zed.Value { v, ok := coerce.ToUint(val) if !ok || (c.max != 0 && v > c.max) { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to "+zson.FormatType(c.typ), val) + return c.zctx.WrapError("cannot cast to "+zson.FormatType(c.typ), val) } return zed.NewUint(c.typ, v) } @@ -95,7 +95,7 @@ type casterBool struct { func (c *casterBool) Eval(ectx Context, val zed.Value) zed.Value { b, ok := coerce.ToBool(val) if !ok { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to bool", val) + return c.zctx.WrapError("cannot cast to bool", val) } return zed.NewBool(b) } @@ -107,7 +107,7 @@ type casterFloat16 struct { func (c *casterFloat16) Eval(ectx Context, val zed.Value) zed.Value { f, ok := coerce.ToFloat(val) if !ok { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to float16", val) + return c.zctx.WrapError("cannot cast to float16", val) } f16 := float16.Fromfloat32(float32(f)) return zed.NewFloat16(f16.Float32()) @@ -120,7 +120,7 @@ type casterFloat32 struct { func (c *casterFloat32) Eval(ectx Context, val zed.Value) zed.Value { f, ok := coerce.ToFloat(val) if !ok { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to float32", val) + return c.zctx.WrapError("cannot cast to float32", val) } return zed.NewFloat32(float32(f)) } @@ -132,7 +132,7 @@ type casterFloat64 struct { func (c *casterFloat64) Eval(ectx Context, val zed.Value) zed.Value { f, ok := coerce.ToFloat(val) if !ok { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to float64", val) + return c.zctx.WrapError("cannot cast to float64", val) } return zed.NewFloat64(f) } @@ -146,13 +146,13 @@ func (c *casterIP) Eval(ectx Context, val zed.Value) zed.Value { return val } if !val.IsString() { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to ip", val) + return c.zctx.WrapError("cannot cast to ip", val) } ip, err := byteconv.ParseIP(val.Bytes()) if err != nil { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to ip", val) + return c.zctx.WrapError("cannot cast to ip", val) } - return ectx.Arena().NewIP(ip) + return zed.NewIP(ip) } type casterNet struct { @@ -164,13 +164,13 @@ func (c *casterNet) Eval(ectx Context, val zed.Value) zed.Value { return val } if !val.IsString() { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to net", val) + return c.zctx.WrapError("cannot cast to net", val) } net, err := netip.ParsePrefix(string(val.Bytes())) if err != nil { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to net", val) + return c.zctx.WrapError("cannot cast to net", val) } - return ectx.Arena().NewNet(net) + return zed.NewNet(net) } type casterDuration struct { @@ -187,7 +187,7 @@ func (c *casterDuration) Eval(ectx Context, val zed.Value) zed.Value { if err != nil { f, ferr := byteconv.ParseFloat64(val.Bytes()) if ferr != nil { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to duration", val) + return c.zctx.WrapError("cannot cast to duration", val) } d = nano.Duration(f) } @@ -198,7 +198,7 @@ func (c *casterDuration) Eval(ectx Context, val zed.Value) zed.Value { } v, ok := coerce.ToInt(val) if !ok { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to duration", val) + return c.zctx.WrapError("cannot cast to duration", val) } return zed.NewDuration(nano.Duration(v)) } @@ -220,7 +220,7 @@ func (c *casterTime) Eval(ectx Context, val zed.Value) zed.Value { if err != nil { v, err := byteconv.ParseFloat64(val.Bytes()) if err != nil { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to time", val) + return c.zctx.WrapError("cannot cast to time", val) } ts = nano.Ts(v) } else { @@ -230,11 +230,11 @@ func (c *casterTime) Eval(ectx Context, val zed.Value) zed.Value { //XXX we call coerce on integers here to avoid unsigned/signed decode v, ok := coerce.ToInt(val) if !ok { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to time: coerce to int failed", val) + return c.zctx.WrapError("cannot cast to time: coerce to int failed", val) } ts = nano.Ts(v) default: - return c.zctx.WrapError(ectx.Arena(), "cannot cast to time", val) + return c.zctx.WrapError("cannot cast to time", val) } return zed.NewTime(ts) } @@ -247,32 +247,32 @@ func (c *casterString) Eval(ectx Context, val zed.Value) zed.Value { id := val.Type().ID() if id == zed.IDBytes { if !utf8.Valid(val.Bytes()) { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to string: invalid UTF-8", val) + return c.zctx.WrapError("cannot cast to string: invalid UTF-8", val) } - return ectx.Arena().New(zed.TypeString, val.Bytes()) + return zed.NewValue(zed.TypeString, val.Bytes()) } if enum, ok := val.Type().(*zed.TypeEnum); ok { selector := zed.DecodeUint(val.Bytes()) symbol, err := enum.Symbol(int(selector)) if err != nil { - return c.zctx.NewError(ectx.Arena(), err) + return c.zctx.NewError(err) } - return ectx.Arena().NewString(symbol) + return zed.NewString(symbol) } if id == zed.IDString { // If it's already stringy, then the Zed encoding can stay // the same and we just update the stringy type. - return ectx.Arena().New(zed.TypeString, val.Bytes()) + return zed.NewValue(zed.TypeString, val.Bytes()) } // Otherwise, we'll use a canonical ZSON value for the string rep // of an arbitrary value cast to a string. - return ectx.Arena().NewString(zson.FormatValue(val)) + return zed.NewString(zson.FormatValue(val)) } type casterBytes struct{} func (c *casterBytes) Eval(ectx Context, val zed.Value) zed.Value { - return ectx.Arena().NewBytes(val.Bytes()) + return zed.NewBytes(val.Bytes()) } type casterNamedType struct { @@ -288,9 +288,9 @@ func (c *casterNamedType) Eval(ectx Context, this zed.Value) zed.Value { } typ, err := c.zctx.LookupTypeNamed(c.name, zed.TypeUnder(val.Type())) if err != nil { - return c.zctx.NewError(ectx.Arena(), err) + return c.zctx.NewError(err) } - return ectx.Arena().New(typ, val.Bytes()) + return zed.NewValue(typ, val.Bytes()) } type casterType struct { @@ -303,14 +303,14 @@ func (c *casterType) Eval(ectx Context, val zed.Value) zed.Value { return val } if id != zed.IDString { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to type", val) + return c.zctx.WrapError("cannot cast to type", val) } - typval, err := zson.ParseValue(c.zctx, ectx.Arena(), val.AsString()) + typval, err := zson.ParseValue(c.zctx, val.AsString()) if err != nil { - return c.zctx.WrapError(ectx.Arena(), err.Error(), val) + return c.zctx.WrapError(err.Error(), val) } if typval.Type().ID() != zed.IDType { - return c.zctx.WrapError(ectx.Arena(), "cannot cast to type", val) + return c.zctx.WrapError("cannot cast to type", val) } return typval } diff --git a/runtime/sam/expr/coerce/coerce.go b/runtime/sam/expr/coerce/coerce.go index cfc9ce302b..569bb48c7c 100644 --- a/runtime/sam/expr/coerce/coerce.go +++ b/runtime/sam/expr/coerce/coerce.go @@ -44,6 +44,7 @@ func ToNumeric[T constraints.Integer | constraints.Float](val zed.Value) T { if val.IsNull() { return 0 } + val = val.Under() switch id := val.Type().ID(); { case zed.IsUnsigned(id): return T(val.Uint()) @@ -56,6 +57,7 @@ func ToNumeric[T constraints.Integer | constraints.Float](val zed.Value) T { } func ToFloat(val zed.Value) (float64, bool) { + val = val.Under() switch id := val.Type().ID(); { case zed.IsUnsigned(id): return float64(val.Uint()), true @@ -71,6 +73,7 @@ func ToFloat(val zed.Value) (float64, bool) { } func ToUint(val zed.Value) (uint64, bool) { + val = val.Under() switch id := val.Type().ID(); { case zed.IsUnsigned(id): return val.Uint(), true @@ -90,6 +93,7 @@ func ToUint(val zed.Value) (uint64, bool) { } func ToInt(val zed.Value) (int64, bool) { + val = val.Under() switch id := val.Type().ID(); { case zed.IsUnsigned(id): return int64(val.Uint()), true @@ -106,6 +110,7 @@ func ToInt(val zed.Value) (int64, bool) { } func ToBool(val zed.Value) (bool, bool) { + val = val.Under() if val.IsString() { v, err := byteconv.ParseBool(val.Bytes()) return v, err == nil diff --git a/runtime/sam/expr/coerce/promote.go b/runtime/sam/expr/coerce/promote.go index 04957ea8f4..31efbbda2e 100644 --- a/runtime/sam/expr/coerce/promote.go +++ b/runtime/sam/expr/coerce/promote.go @@ -11,6 +11,7 @@ var ErrIncompatibleTypes = errors.New("incompatible types") var ErrOverflow = errors.New("integer overflow: uint64 value too large for int64") func Promote(a, b zed.Value) (int, error) { + a, b = a.Under(), b.Under() aid, bid := a.Type().ID(), b.Type().ID() switch { case aid == bid: diff --git a/runtime/sam/expr/context.go b/runtime/sam/expr/context.go index 873d831cc5..24331162fc 100644 --- a/runtime/sam/expr/context.go +++ b/runtime/sam/expr/context.go @@ -9,31 +9,19 @@ import ( // of Zed values both for let-style temporary variables accessible via // the scope and for allocating results. type Context interface { - Arena() *zed.Arena - // Vars() accesses the variables reachable in the current scope. Vars() []zed.Value + zed.Allocator } -type Ctx struct { - arena *zed.Arena - vars []zed.Value -} - -var _ Context = (*Ctx)(nil) +type allocator struct{} -func NewContext(arena *zed.Arena) *Ctx { - return NewContextWithVars(arena, nil) -} - -func NewContextWithVars(arena *zed.Arena, vars []zed.Value) *Ctx { - return &Ctx{arena, vars} -} +var _ Context = (*allocator)(nil) -func (a *Ctx) Arena() *zed.Arena { - return a.arena +func NewContext() *allocator { + return &allocator{} } -func (a *Ctx) Vars() []zed.Value { - return a.vars +func (*allocator) Vars() []zed.Value { + return nil } diff --git a/runtime/sam/expr/cutter.go b/runtime/sam/expr/cutter.go index 9a3d7054e0..6cb7aa1327 100644 --- a/runtime/sam/expr/cutter.go +++ b/runtime/sam/expr/cutter.go @@ -56,7 +56,7 @@ func (c *Cutter) FoundCut() bool { func (c *Cutter) Eval(ectx Context, in zed.Value) zed.Value { rb, paths, err := c.lookupBuilder(ectx, in) if err != nil { - return c.zctx.WrapError(ectx.Arena(), fmt.Sprintf("cut: %s", err), in) + return c.zctx.WrapError(fmt.Sprintf("cut: %s", err), in) } types := c.typeCache rb.Reset() @@ -81,7 +81,7 @@ func (c *Cutter) Eval(ectx Context, in zed.Value) zed.Value { if err != nil { panic(err) } - rec := ectx.Arena().New(rb.Type(c.outTypes.Lookup(types), types), bytes) + rec := zed.NewValue(rb.Type(c.outTypes.Lookup(types), types), bytes) for _, d := range droppers { rec = d.Eval(ectx, rec) } diff --git a/runtime/sam/expr/dot.go b/runtime/sam/expr/dot.go index 000c6f75bb..869be65c70 100644 --- a/runtime/sam/expr/dot.go +++ b/runtime/sam/expr/dot.go @@ -39,21 +39,21 @@ func NewDottedExpr(zctx *zed.Context, f field.Path) Evaluator { } func (d *DotExpr) Eval(ectx Context, this zed.Value) zed.Value { - val := d.record.Eval(ectx, this).Under(ectx.Arena()) + val := d.record.Eval(ectx, this).Under() // Cases are ordered by decreasing expected frequency. switch typ := val.Type().(type) { case *zed.TypeRecord: i, ok := d.fieldIndex(typ) if !ok { - return d.zctx.Missing(ectx.Arena()) + return d.zctx.Missing() } - return ectx.Arena().New(typ.Fields[i].Type, getNthFromContainer(val.Bytes(), i)) + return zed.NewValue(typ.Fields[i].Type, getNthFromContainer(val.Bytes(), i)) case *zed.TypeMap: - return indexMap(d.zctx, ectx, typ, val.Bytes(), ectx.Arena().NewString(d.field)) + return indexMap(d.zctx, ectx, typ, val.Bytes(), zed.NewString(d.field)) case *zed.TypeOfType: return d.evalTypeOfType(ectx, val.Bytes()) } - return d.zctx.Missing(ectx.Arena()) + return d.zctx.Missing() } func (d *DotExpr) fieldIndex(typ *zed.TypeRecord) (int, bool) { @@ -79,10 +79,10 @@ func (d *DotExpr) evalTypeOfType(ectx Context, b zcode.Bytes) zed.Value { typ, _ := d.zctx.DecodeTypeValue(b) if typ, ok := zed.TypeUnder(typ).(*zed.TypeRecord); ok { if typ, ok := typ.TypeOfField(d.field); ok { - return d.zctx.LookupTypeValue(ectx.Arena(), typ) + return d.zctx.LookupTypeValue(typ) } } - return d.zctx.Missing(ectx.Arena()) + return d.zctx.Missing() } // DotExprToString returns Zed for the Evaluator assuming it's a field expr. diff --git a/runtime/sam/expr/dropper.go b/runtime/sam/expr/dropper.go index b8dfb2f53b..5286ac06c5 100644 --- a/runtime/sam/expr/dropper.go +++ b/runtime/sam/expr/dropper.go @@ -27,7 +27,7 @@ func (d *dropper) drop(ectx Context, in zed.Value) zed.Value { if err != nil { panic(err) } - return ectx.Arena().New(d.typ, val) + return zed.NewValue(d.typ, val) } type Dropper struct { @@ -108,7 +108,7 @@ func (d *Dropper) Eval(ectx Context, in zed.Value) zed.Value { d.droppers[id] = dropper } if dropper == nil { - return d.zctx.Quiet(ectx.Arena()) + return d.zctx.Quiet() } return dropper.drop(ectx, in) } diff --git a/runtime/sam/expr/eval.go b/runtime/sam/expr/eval.go index a757cca3be..b8ebf8e617 100644 --- a/runtime/sam/expr/eval.go +++ b/runtime/sam/expr/eval.go @@ -19,7 +19,7 @@ type Evaluator interface { } type Function interface { - Call(Context, []zed.Value) zed.Value + Call(zed.Allocator, []zed.Value) zed.Value } type Not struct { @@ -75,7 +75,7 @@ func EvalBool(zctx *zed.Context, ectx Context, this zed.Value, e Evaluator) (zed if val.IsError() { return val, false } - return zctx.WrapError(ectx.Arena(), "not type bool", val), false + return zctx.WrapError("not type bool", val), false } func (a *And) Eval(ectx Context, this zed.Value) zed.Value { @@ -137,10 +137,8 @@ func (i *In) Eval(ectx Context, this zed.Value) zed.Value { if container.IsError() { return container } - arena := ectx.Arena() - elem = elem.Under(arena) err := container.Walk(func(typ zed.Type, body zcode.Bytes) error { - if coerce.Equal(elem, arena.New(typ, body).Under(arena)) { + if coerce.Equal(elem, zed.NewValue(typ, body)) { return errMatch } return nil @@ -151,7 +149,7 @@ func (i *In) Eval(ectx Context, this zed.Value) zed.Value { case nil: return zed.False default: - return i.zctx.NewError(arena, err) + return i.zctx.NewError(err) } } @@ -225,11 +223,11 @@ func (n *numeric) evalAndPromote(ectx Context, this zed.Value) (zed.Value, zed.V } id, err := coerce.Promote(lhsVal, rhsVal) if err != nil { - return zed.Null, zed.Null, nil, n.zctx.NewError(ectx.Arena(), err).Ptr() + return zed.Null, zed.Null, nil, n.zctx.NewError(err).Ptr() } typ, err := n.zctx.LookupType(id) if err != nil { - return zed.Null, zed.Null, nil, n.zctx.NewError(ectx.Arena(), err).Ptr() + return zed.Null, zed.Null, nil, n.zctx.NewError(err).Ptr() } return lhsVal, rhsVal, typ, nil } @@ -243,15 +241,13 @@ func (n *numeric) eval(ectx Context, this zed.Value) (zed.Value, zed.Value, *zed if rhs.IsError() { return zed.Null, zed.Null, &rhs } - arena := ectx.Arena() - lhs, rhs = lhs.Under(arena), rhs.Under(arena) - return enumToIndex(arena, lhs), enumToIndex(arena, rhs), nil + return enumToIndex(ectx, lhs), enumToIndex(ectx, rhs), nil } // enumToIndex converts an enum to its index value. -func enumToIndex(arena *zed.Arena, val zed.Value) zed.Value { +func enumToIndex(ectx Context, val zed.Value) zed.Value { if _, ok := val.Type().(*zed.TypeEnum); ok { - return arena.New(zed.TypeUint64, val.Bytes()) + return zed.NewValue(zed.TypeUint64, val.Bytes()) } return val } @@ -292,8 +288,7 @@ func (c *Compare) Eval(ectx Context, this zed.Value) zed.Value { if rhs.IsError() { return rhs } - arena := ectx.Arena() - lhs, rhs = lhs.Under(arena), rhs.Under(arena) + lhs, rhs = lhs.Under(), rhs.Under() if lhs.IsNull() { if rhs.IsNull() { @@ -421,9 +416,9 @@ func (a *Add) Eval(ectx Context, this zed.Value) zed.Value { return zed.NewFloat(typ, toFloat(lhsVal)+toFloat(rhsVal)) case id == zed.IDString: v1, v2 := zed.DecodeString(lhsVal.Bytes()), zed.DecodeString(rhsVal.Bytes()) - return ectx.Arena().New(typ, zed.EncodeString(v1+v2)) + return zed.NewValue(typ, zed.EncodeString(v1+v2)) } - return a.zctx.NewErrorf(ectx.Arena(), "type %s incompatible with '+' operator", zson.FormatType(typ)) + return a.zctx.NewErrorf("type %s incompatible with '+' operator", zson.FormatType(typ)) } func (s *Subtract) Eval(ectx Context, this zed.Value) zed.Value { @@ -443,7 +438,7 @@ func (s *Subtract) Eval(ectx Context, this zed.Value) zed.Value { case zed.IsFloat(id): return zed.NewFloat(typ, toFloat(lhsVal)-toFloat(rhsVal)) } - return s.zctx.NewErrorf(ectx.Arena(), "type %s incompatible with '-' operator", zson.FormatType(typ)) + return s.zctx.NewErrorf("type %s incompatible with '-' operator", zson.FormatType(typ)) } func (m *Multiply) Eval(ectx Context, this zed.Value) zed.Value { @@ -459,7 +454,7 @@ func (m *Multiply) Eval(ectx Context, this zed.Value) zed.Value { case zed.IsFloat(id): return zed.NewFloat(typ, toFloat(lhsVal)*toFloat(rhsVal)) } - return m.zctx.NewErrorf(ectx.Arena(), "type %s incompatible with '*' operator", zson.FormatType(typ)) + return m.zctx.NewErrorf("type %s incompatible with '*' operator", zson.FormatType(typ)) } func (d *Divide) Eval(ectx Context, this zed.Value) zed.Value { @@ -471,23 +466,23 @@ func (d *Divide) Eval(ectx Context, this zed.Value) zed.Value { case zed.IsUnsigned(id): v := toUint(rhsVal) if v == 0 { - return d.zctx.NewError(ectx.Arena(), DivideByZero) + return d.zctx.NewError(DivideByZero) } return zed.NewUint(typ, toUint(lhsVal)/v) case zed.IsSigned(id): v := toInt(rhsVal) if v == 0 { - return d.zctx.NewError(ectx.Arena(), DivideByZero) + return d.zctx.NewError(DivideByZero) } return zed.NewInt(typ, toInt(lhsVal)/v) case zed.IsFloat(id): v := toFloat(rhsVal) if v == 0 { - return d.zctx.NewError(ectx.Arena(), DivideByZero) + return d.zctx.NewError(DivideByZero) } return zed.NewFloat(typ, toFloat(lhsVal)/v) } - return d.zctx.NewErrorf(ectx.Arena(), "type %s incompatible with '/' operator", zson.FormatType(typ)) + return d.zctx.NewErrorf("type %s incompatible with '/' operator", zson.FormatType(typ)) } func (m *Modulo) Eval(ectx Context, this zed.Value) zed.Value { @@ -499,17 +494,17 @@ func (m *Modulo) Eval(ectx Context, this zed.Value) zed.Value { case zed.IsUnsigned(id): v := toUint(rhsVal) if v == 0 { - return m.zctx.NewError(ectx.Arena(), DivideByZero) + return m.zctx.NewError(DivideByZero) } return zed.NewUint(typ, lhsVal.Uint()%v) case zed.IsSigned(id): v := toInt(rhsVal) if v == 0 { - return m.zctx.NewError(ectx.Arena(), DivideByZero) + return m.zctx.NewError(DivideByZero) } return zed.NewInt(typ, toInt(lhsVal)%v) } - return m.zctx.NewErrorf(ectx.Arena(), "type %s incompatible with '%%' operator", zson.FormatType(typ)) + return m.zctx.NewErrorf("type %s incompatible with '%%' operator", zson.FormatType(typ)) } type UnaryMinus struct { @@ -536,53 +531,53 @@ func (u *UnaryMinus) Eval(ectx Context, this zed.Value) zed.Value { case zed.IDInt8: v := val.Int() if v == math.MinInt8 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' underflow", val) + return u.zctx.WrapError("unary '-' underflow", val) } return zed.NewInt8(int8(-v)) case zed.IDInt16: v := val.Int() if v == math.MinInt16 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' underflow", val) + return u.zctx.WrapError("unary '-' underflow", val) } return zed.NewInt16(int16(-v)) case zed.IDInt32: v := val.Int() if v == math.MinInt32 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' underflow", val) + return u.zctx.WrapError("unary '-' underflow", val) } return zed.NewInt32(int32(-v)) case zed.IDInt64: v := val.Int() if v == math.MinInt64 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' underflow", val) + return u.zctx.WrapError("unary '-' underflow", val) } return zed.NewInt64(-v) case zed.IDUint8: v := val.Uint() if v > math.MaxInt8 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' overflow", val) + return u.zctx.WrapError("unary '-' overflow", val) } return zed.NewInt8(int8(-v)) case zed.IDUint16: v := val.Uint() if v > math.MaxInt16 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' overflow", val) + return u.zctx.WrapError("unary '-' overflow", val) } return zed.NewInt16(int16(-v)) case zed.IDUint32: v := val.Uint() if v > math.MaxInt32 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' overflow", val) + return u.zctx.WrapError("unary '-' overflow", val) } return zed.NewInt32(int32(-v)) case zed.IDUint64: v := val.Uint() if v > math.MaxInt64 { - return u.zctx.WrapError(ectx.Arena(), "unary '-' overflow", val) + return u.zctx.WrapError("unary '-' overflow", val) } return zed.NewInt64(int64(-v)) } - return u.zctx.WrapError(ectx.Arena(), "type incompatible with unary '-' operator", val) + return u.zctx.WrapError("type incompatible with unary '-' operator", val) } func getNthFromContainer(container zcode.Bytes, idx int) zcode.Bytes { @@ -639,14 +634,14 @@ func (i *Index) Eval(ectx Context, this zed.Value) zed.Value { case *zed.TypeMap: return indexMap(i.zctx, ectx, typ, container.Bytes(), index) default: - return i.zctx.Missing(ectx.Arena()) + return i.zctx.Missing() } } func indexVector(zctx *zed.Context, ectx Context, inner zed.Type, vector zcode.Bytes, index zed.Value) zed.Value { id := index.Type().ID() if !zed.IsInteger(id) { - return zctx.WrapError(ectx.Arena(), "array index is not an integer", index) + return zctx.WrapError("array index is not an integer", index) } var idx int if zed.IsSigned(id) { @@ -656,7 +651,7 @@ func indexVector(zctx *zed.Context, ectx Context, inner zed.Type, vector zcode.B } zv := getNthFromContainer(vector, idx) if zv == nil { - return zctx.Missing(ectx.Arena()) + return zctx.Missing() } return deunion(ectx, inner, zv) } @@ -664,20 +659,19 @@ func indexVector(zctx *zed.Context, ectx Context, inner zed.Type, vector zcode.B func indexRecord(zctx *zed.Context, ectx Context, typ *zed.TypeRecord, record zcode.Bytes, index zed.Value) zed.Value { id := index.Type().ID() if id != zed.IDString { - return zctx.WrapError(ectx.Arena(), "record index is not a string", index) + return zctx.WrapError("record index is not a string", index) } field := zed.DecodeString(index.Bytes()) - arena := ectx.Arena() - val := arena.New(typ, record).Ptr().Deref(arena, field) + val := zed.NewValue(typ, record).Ptr().Deref(field) if val == nil { - return zctx.Missing(arena) + return zctx.Missing() } return *val } func indexMap(zctx *zed.Context, ectx Context, typ *zed.TypeMap, mapBytes zcode.Bytes, key zed.Value) zed.Value { if key.IsMissing() { - return zctx.Missing(ectx.Arena()) + return zctx.Missing() } if key.Type() != typ.KeyType { if union, ok := zed.TypeUnder(typ.KeyType).(*zed.TypeUnion); ok { @@ -689,19 +683,19 @@ func indexMap(zctx *zed.Context, ectx Context, typ *zed.TypeMap, mapBytes zcode. } } } - return zctx.Missing(ectx.Arena()) + return zctx.Missing() } if valBytes, ok := lookupKey(mapBytes, key.Bytes()); ok { return deunion(ectx, typ.ValType, valBytes) } - return zctx.Missing(ectx.Arena()) + return zctx.Missing() } func deunion(ectx Context, typ zed.Type, b zcode.Bytes) zed.Value { if union, ok := typ.(*zed.TypeUnion); ok { typ, b = union.Untag(b) } - return ectx.Arena().New(typ, b) + return zed.NewValue(typ, b) } type Conditional struct { @@ -723,7 +717,7 @@ func NewConditional(zctx *zed.Context, predicate, thenExpr, elseExpr Evaluator) func (c *Conditional) Eval(ectx Context, this zed.Value) zed.Value { val := c.predicate.Eval(ectx, this) if val.Type().ID() != zed.IDBool { - return c.zctx.WrapError(ectx.Arena(), "?-operator: bool predicate required", val) + return c.zctx.WrapError("?-operator: bool predicate required", val) } if val.Bool() { return c.thenExpr.Eval(ectx, this) diff --git a/runtime/sam/expr/extent/span.go b/runtime/sam/expr/extent/span.go index 4f730e9656..e9dc79d8fa 100644 --- a/runtime/sam/expr/extent/span.go +++ b/runtime/sam/expr/extent/span.go @@ -32,7 +32,6 @@ type Generic struct { first zed.Value last zed.Value cmp expr.CompareFn - arena *zed.Arena } // Create a new Range from generic range of zed.Values according @@ -42,12 +41,10 @@ func NewGeneric(lower, upper zed.Value, cmp expr.CompareFn) *Generic { if cmp(lower, upper) > 0 { lower, upper = upper, lower } - arena := zed.NewArena() return &Generic{ - first: lower.Copy(arena), - last: upper.Copy(arena), + first: lower, + last: upper, cmp: cmp, - arena: arena, } } @@ -94,9 +91,9 @@ func (g *Generic) Crop(s Span) bool { func (g *Generic) Extend(val zed.Value) { if g.cmp(val, g.first) < 0 { - g.first = val.Copy(g.arena) + g.first = val.Copy() } else if g.cmp(val, g.last) > 0 { - g.last = val.Copy(g.arena) + g.last = val.Copy() } } diff --git a/runtime/sam/expr/filter.go b/runtime/sam/expr/filter.go index 4efd6d9901..6b5427b86d 100644 --- a/runtime/sam/expr/filter.go +++ b/runtime/sam/expr/filter.go @@ -33,10 +33,10 @@ func (s *searchByPred) Eval(ectx Context, val zed.Value) zed.Value { } } if errMatch == val.Walk(func(typ zed.Type, body zcode.Bytes) error { - if s.searchType(ectx, typ) { + if s.searchType(typ) { return errMatch } - if s.pred(ectx.Arena().New(typ, body)) { + if s.pred(zed.NewValue(typ, body)) { return errMatch } return nil @@ -46,7 +46,7 @@ func (s *searchByPred) Eval(ectx Context, val zed.Value) zed.Value { return zed.False } -func (s *searchByPred) searchType(ectx Context, typ zed.Type) bool { +func (s *searchByPred) searchType(typ zed.Type) bool { if match, ok := s.types[typ]; ok { return match } @@ -56,7 +56,7 @@ func (s *searchByPred) searchType(ectx Context, typ zed.Type) bool { var nameIter FieldNameIter nameIter.Init(recType) for !nameIter.Done() { - if s.pred(ectx.Arena().NewString(string(nameIter.Next()))) { + if s.pred(zed.NewString(string(nameIter.Next()))) { match = true break } @@ -129,7 +129,7 @@ func (s *search) Eval(ectx Context, val zed.Value) zed.Value { } return nil } - if s.compare(ectx.Arena().New(typ, body)) { + if s.compare(zed.NewValue(typ, body)) { return errMatch } return nil @@ -262,7 +262,7 @@ func (f *filterApplier) Eval(ectx Context, this zed.Value) zed.Value { if val.Bool() { return this } - return f.zctx.Missing(ectx.Arena()) + return f.zctx.Missing() } return val } diff --git a/runtime/sam/expr/filter_test.go b/runtime/sam/expr/filter_test.go index 75d659075b..387d5f765d 100644 --- a/runtime/sam/expr/filter_test.go +++ b/runtime/sam/expr/filter_test.go @@ -1,7 +1,6 @@ package expr_test import ( - "context" "encoding/binary" "encoding/hex" "testing" @@ -46,9 +45,7 @@ func runCasesHelper(t *testing.T, record string, cases []testcase, expectBufferF t.Helper() zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - rec, err := zson.ParseValue(zctx, arena, record) + rec, err := zson.ParseValue(zctx, record) require.NoError(t, err, "record: %q", record) for _, c := range cases { @@ -56,8 +53,7 @@ func runCasesHelper(t *testing.T, record string, cases []testcase, expectBufferF t.Helper() p, _, err := compiler.Parse(c.filter) require.NoError(t, err, "filter: %q", c.filter) - rctx := runtime.NewContext(context.Background(), zctx) - job, err := compiler.NewJob(rctx, p, nil, nil) + job, err := compiler.NewJob(runtime.DefaultContext(), p, nil, nil) require.NoError(t, err, "filter: %q", c.filter) err = job.Optimize() require.NoError(t, err, "filter: %q", c.filter) @@ -69,7 +65,7 @@ func runCasesHelper(t *testing.T, record string, cases []testcase, expectBufferF f, err := filterMaker.AsEvaluator() assert.NoError(t, err, "filter: %q", c.filter) if f != nil { - assert.Equal(t, c.expected, filter(expr.NewContext(arena), rec, f), + assert.Equal(t, c.expected, filter(expr.NewContext(), rec, f), "filter: %q\nrecord: %s", c.filter, zson.FormatValue(rec)) } bf, err := filterMaker.AsBufferFilter() diff --git a/runtime/sam/expr/flattener.go b/runtime/sam/expr/flattener.go index d007dca9ee..eb890921ce 100644 --- a/runtime/sam/expr/flattener.go +++ b/runtime/sam/expr/flattener.go @@ -60,7 +60,7 @@ func recode(dst zcode.Bytes, typ *zed.TypeRecord, in zcode.Bytes) (zcode.Bytes, return dst, nil } -func (f *Flattener) Flatten(arena *zed.Arena, r zed.Value) (zed.Value, error) { +func (f *Flattener) Flatten(r zed.Value) (zed.Value, error) { id := r.Type().ID() flatType := f.mapper.Lookup(id) if flatType == nil { @@ -77,7 +77,7 @@ func (f *Flattener) Flatten(arena *zed.Arena, r zed.Value) (zed.Value, error) { if err != nil { return zed.Null, err } - return arena.New(flatType.(*zed.TypeRecord), zv), nil + return zed.NewValue(flatType.(*zed.TypeRecord), zv), nil } // FlattenFields turns nested records into a series of fields of diff --git a/runtime/sam/expr/function/bytes.go b/runtime/sam/expr/function/bytes.go index a775cccf25..5454c244f2 100644 --- a/runtime/sam/expr/function/bytes.go +++ b/runtime/sam/expr/function/bytes.go @@ -5,7 +5,6 @@ import ( "encoding/hex" "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" ) // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#base64 @@ -13,26 +12,25 @@ type Base64 struct { zctx *zed.Context } -func (b *Base64) Call(ectx expr.Context, args []zed.Value) zed.Value { - arena := ectx.Arena() - val := args[0].Under(arena) +func (b *Base64) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() switch val.Type().ID() { case zed.IDBytes: if val.IsNull() { - return b.zctx.NewErrorf(ectx.Arena(), "base64: illegal null argument") + return b.zctx.NewErrorf("base64: illegal null argument") } - return arena.NewString(base64.StdEncoding.EncodeToString(val.Bytes())) + return zed.NewString(base64.StdEncoding.EncodeToString(val.Bytes())) case zed.IDString: if val.IsNull() { return zed.NullBytes } bytes, err := base64.StdEncoding.DecodeString(zed.DecodeString(val.Bytes())) if err != nil { - return b.zctx.WrapError(ectx.Arena(), "base64: string argument is not base64", val) + return b.zctx.WrapError("base64: string argument is not base64", val) } - return arena.NewBytes(bytes) + return zed.NewBytes(bytes) default: - return b.zctx.WrapError(ectx.Arena(), "base64: argument must a bytes or string type", val) + return b.zctx.WrapError("base64: argument must a bytes or string type", val) } } @@ -41,25 +39,24 @@ type Hex struct { zctx *zed.Context } -func (h *Hex) Call(ectx expr.Context, args []zed.Value) zed.Value { - arena := ectx.Arena() - val := args[0].Under(arena) +func (h *Hex) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() switch val.Type().ID() { case zed.IDBytes: if val.IsNull() { - return h.zctx.NewErrorf(ectx.Arena(), "hex: illegal null argument") + return h.zctx.NewErrorf("hex: illegal null argument") } - return arena.NewString(hex.EncodeToString(val.Bytes())) + return zed.NewString(hex.EncodeToString(val.Bytes())) case zed.IDString: if val.IsNull() { return zed.NullBytes } b, err := hex.DecodeString(zed.DecodeString(val.Bytes())) if err != nil { - return h.zctx.WrapError(ectx.Arena(), "hex: string argument is not hexidecimal", val) + return h.zctx.WrapError("hex: string argument is not hexidecimal", val) } - return arena.NewBytes(b) + return zed.NewBytes(b) default: - return h.zctx.WrapError(ectx.Arena(), "base64: argument must a bytes or string type", val) + return h.zctx.WrapError("base64: argument must a bytes or string type", val) } } diff --git a/runtime/sam/expr/function/coalesce.go b/runtime/sam/expr/function/coalesce.go index 2ad7784aa6..d770299e51 100644 --- a/runtime/sam/expr/function/coalesce.go +++ b/runtime/sam/expr/function/coalesce.go @@ -1,14 +1,11 @@ package function -import ( - "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" -) +import "github.com/brimdata/zed" // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#coalesce type Coalesce struct{} -func (c *Coalesce) Call(_ expr.Context, args []zed.Value) zed.Value { +func (c *Coalesce) Call(_ zed.Allocator, args []zed.Value) zed.Value { for i := range args { val := &args[i] if !val.IsNull() && !val.IsMissing() && !val.IsQuiet() { diff --git a/runtime/sam/expr/function/compare.go b/runtime/sam/expr/function/compare.go index 9a065b2cde..97f6a5e643 100644 --- a/runtime/sam/expr/function/compare.go +++ b/runtime/sam/expr/function/compare.go @@ -20,11 +20,11 @@ func NewCompare(zctx *zed.Context) *Compare { } } -func (e *Compare) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (e *Compare) Call(_ zed.Allocator, args []zed.Value) zed.Value { nullsMax := true if len(args) == 3 { if zed.TypeUnder(args[2].Type()) != zed.TypeBool { - return e.zctx.WrapError(ectx.Arena(), "compare: nullsMax arg is not bool", args[2]) + return e.zctx.WrapError("compare: nullsMax arg is not bool", args[2]) } nullsMax = args[2].Bool() } diff --git a/runtime/sam/expr/function/fields.go b/runtime/sam/expr/function/fields.go index 0728825bdc..b1866d22ae 100644 --- a/runtime/sam/expr/function/fields.go +++ b/runtime/sam/expr/function/fields.go @@ -2,7 +2,6 @@ package function import ( "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" ) @@ -34,16 +33,15 @@ func buildPath(typ *zed.TypeRecord, b *zcode.Builder, prefix []string) { } } -func (f *Fields) Call(ectx expr.Context, args []zed.Value) zed.Value { - arena := ectx.Arena() - subjectVal := args[0].Under(arena) +func (f *Fields) Call(_ zed.Allocator, args []zed.Value) zed.Value { + subjectVal := args[0].Under() typ := f.recordType(subjectVal) if typ == nil { - return f.zctx.Missing(ectx.Arena()) + return f.zctx.Missing() } var b zcode.Builder buildPath(typ, &b, nil) - return arena.New(f.typ, b.Bytes()) + return zed.NewValue(f.typ, b.Bytes()) } func (f *Fields) recordType(val zed.Value) *zed.TypeRecord { diff --git a/runtime/sam/expr/function/flatten.go b/runtime/sam/expr/function/flatten.go index 20c0db50d3..470f68fd6f 100644 --- a/runtime/sam/expr/function/flatten.go +++ b/runtime/sam/expr/function/flatten.go @@ -3,7 +3,6 @@ package function import ( "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/field" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" ) @@ -26,7 +25,7 @@ func NewFlatten(zctx *zed.Context) *Flatten { } } -func (n *Flatten) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (n *Flatten) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] typ := zed.TypeRecordOf(val.Type()) if typ == nil { @@ -35,7 +34,7 @@ func (n *Flatten) Call(ectx expr.Context, args []zed.Value) zed.Value { inner := n.innerTypeOf(val.Bytes(), typ.Fields) n.Reset() n.encode(typ.Fields, inner, field.Path{}, val.Bytes()) - return ectx.Arena().New(n.zctx.LookupTypeArray(inner), n.Bytes()) + return zed.NewValue(n.zctx.LookupTypeArray(inner), n.Bytes()) } func (n *Flatten) innerTypeOf(b zcode.Bytes, fields []zed.Field) zed.Type { diff --git a/runtime/sam/expr/function/function.go b/runtime/sam/expr/function/function.go index 56b5dcdf2f..39a32ba049 100644 --- a/runtime/sam/expr/function/function.go +++ b/runtime/sam/expr/function/function.go @@ -197,9 +197,9 @@ func HasBoolResult(name string) bool { return false } -func underAll(arena *zed.Arena, args []zed.Value) []zed.Value { +func underAll(args []zed.Value) []zed.Value { for i := range args { - args[i] = args[i].Under(arena) + args[i] = args[i].Under() } return args } diff --git a/runtime/sam/expr/function/grep.go b/runtime/sam/expr/function/grep.go index f01890440c..c6b2c80b13 100644 --- a/runtime/sam/expr/function/grep.go +++ b/runtime/sam/expr/function/grep.go @@ -12,15 +12,15 @@ type Grep struct { zctx *zed.Context } -func (g *Grep) Call(ectx expr.Context, vals []zed.Value) zed.Value { +func (g *Grep) Call(_ zed.Allocator, vals []zed.Value) zed.Value { patternVal, inputVal := vals[0], vals[1] if zed.TypeUnder(patternVal.Type()) != zed.TypeString { - return g.zctx.WrapError(ectx.Arena(), "grep(): pattern argument must be a string", patternVal) + return g.zctx.WrapError("grep(): pattern argument must be a string", patternVal) } if p := patternVal.AsString(); g.grep == nil || g.pattern != p { g.pattern = p term := norm.NFC.Bytes(patternVal.Bytes()) g.grep = expr.NewSearchString(string(term), nil) } - return g.grep.Eval(ectx, inputVal) + return g.grep.Eval(expr.NewContext(), inputVal) } diff --git a/runtime/sam/expr/function/grok.go b/runtime/sam/expr/function/grok.go index f6ac909c4a..f6b5f9e7aa 100644 --- a/runtime/sam/expr/function/grok.go +++ b/runtime/sam/expr/function/grok.go @@ -5,7 +5,6 @@ import ( "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/grok" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" ) @@ -24,31 +23,30 @@ func newGrok(zctx *zed.Context) *Grok { } } -func (g *Grok) Call(ectx expr.Context, args []zed.Value) zed.Value { - arena := ectx.Arena() +func (g *Grok) Call(_ zed.Allocator, args []zed.Value) zed.Value { patternArg, inputArg, defArg := args[0], args[1], zed.NullString if len(args) == 3 { defArg = args[2] } switch { case zed.TypeUnder(defArg.Type()) != zed.TypeString: - return g.error(arena, "definitions argument must be a string", defArg) + return g.error("definitions argument must be a string", defArg) case zed.TypeUnder(patternArg.Type()) != zed.TypeString: - return g.error(arena, "pattern argument must be a string", patternArg) + return g.error("pattern argument must be a string", patternArg) case zed.TypeUnder(inputArg.Type()) != zed.TypeString: - return g.error(arena, "input argument must be a string", inputArg) + return g.error("input argument must be a string", inputArg) } h, err := g.getHost(defArg.AsString()) if err != nil { - return g.error(arena, err.Error(), defArg) + return g.error(err.Error(), defArg) } p, err := h.getPattern(patternArg.AsString()) if err != nil { - return g.error(arena, err.Error(), patternArg) + return g.error(err.Error(), patternArg) } keys, vals := p.ParseKeyValues(inputArg.AsString()) if vals == nil { - return g.error(arena, "value does not match pattern", inputArg) + return g.error("value does not match pattern", inputArg) } g.fields = g.fields[:0] for _, key := range keys { @@ -59,11 +57,11 @@ func (g *Grok) Call(ectx expr.Context, args []zed.Value) zed.Value { for _, s := range vals { g.builder.Append([]byte(s)) } - return arena.New(typ, g.builder.Bytes()) + return zed.NewValue(typ, g.builder.Bytes()) } -func (g *Grok) error(arena *zed.Arena, msg string, val zed.Value) zed.Value { - return g.zctx.WrapError(arena, "grok(): "+msg, val) +func (g *Grok) error(msg string, val zed.Value) zed.Value { + return g.zctx.WrapError("grok(): "+msg, val) } func (g *Grok) getHost(defs string) (*host, error) { diff --git a/runtime/sam/expr/function/has.go b/runtime/sam/expr/function/has.go index 413acb604e..8ec011bd39 100644 --- a/runtime/sam/expr/function/has.go +++ b/runtime/sam/expr/function/has.go @@ -1,14 +1,11 @@ package function -import ( - "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" -) +import "github.com/brimdata/zed" // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#has type Has struct{} -func (h *Has) Call(_ expr.Context, args []zed.Value) zed.Value { +func (h *Has) Call(_ zed.Allocator, args []zed.Value) zed.Value { for _, val := range args { if val.IsError() { if val.IsMissing() || val.IsQuiet() { @@ -25,7 +22,7 @@ type Missing struct { has Has } -func (m *Missing) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (m *Missing) Call(ectx zed.Allocator, args []zed.Value) zed.Value { val := m.has.Call(ectx, args) if val.Type() == zed.TypeBool { return zed.NewBool(!val.Bool()) diff --git a/runtime/sam/expr/function/ip.go b/runtime/sam/expr/function/ip.go index 599fdec70a..5041eca6ee 100644 --- a/runtime/sam/expr/function/ip.go +++ b/runtime/sam/expr/function/ip.go @@ -5,7 +5,6 @@ import ( "net/netip" "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" "github.com/brimdata/zed/zson" ) @@ -15,17 +14,17 @@ type NetworkOf struct { zctx *zed.Context } -func (n *NetworkOf) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (n *NetworkOf) Call(_ zed.Allocator, args []zed.Value) zed.Value { id := args[0].Type().ID() if id != zed.IDIP { - return n.zctx.WrapError(ectx.Arena(), "network_of: not an IP", args[0]) + return n.zctx.WrapError("network_of: not an IP", args[0]) } ip := zed.DecodeIP(args[0].Bytes()) var bits int if len(args) == 1 { switch { case !ip.Is4(): - return n.zctx.WrapError(ectx.Arena(), "network_of: not an IPv4 address", args[0]) + return n.zctx.WrapError("network_of: not an IPv4 address", args[0]) case ip.As4()[0] < 0x80: bits = 8 case ip.As4()[0] < 0xc0: @@ -40,12 +39,11 @@ func (n *NetworkOf) Call(ectx expr.Context, args []zed.Value) zed.Value { case id == zed.IDIP: mask := zed.DecodeIP(body) if mask.BitLen() != ip.BitLen() { - arena := ectx.Arena() - return n.zctx.WrapError(arena, "network_of: address and mask have different lengths", n.addressAndMask(arena, args[0], args[1])) + return n.zctx.WrapError("network_of: address and mask have different lengths", addressAndMask(args[0], args[1])) } bits = zed.LeadingOnes(mask.AsSlice()) if netip.PrefixFrom(mask, bits).Masked().Addr() != mask { - return n.zctx.WrapError(ectx.Arena(), "network_of: mask is non-contiguous", args[1]) + return n.zctx.WrapError("network_of: mask is non-contiguous", args[1]) } case zed.IsInteger(id): if zed.IsSigned(id) { @@ -54,20 +52,19 @@ func (n *NetworkOf) Call(ectx expr.Context, args []zed.Value) zed.Value { bits = int(args[1].Uint()) } if bits > 128 || bits > 32 && ip.Is4() { - arena := ectx.Arena() - return n.zctx.WrapError(arena, "network_of: CIDR bit count out of range", n.addressAndMask(arena, args[0], args[1])) + return n.zctx.WrapError("network_of: CIDR bit count out of range", addressAndMask(args[0], args[1])) } default: - return n.zctx.WrapError(ectx.Arena(), "network_of: bad arg for CIDR mask", args[1]) + return n.zctx.WrapError("network_of: bad arg for CIDR mask", args[1]) } } // Mask for canonical form. prefix := netip.PrefixFrom(ip, bits).Masked() - return ectx.Arena().NewNet(prefix) + return zed.NewNet(prefix) } -func (n *NetworkOf) addressAndMask(arena *zed.Arena, address, mask zed.Value) zed.Value { - val, err := zson.MarshalZNG(n.zctx, arena, struct { +func addressAndMask(address, mask zed.Value) zed.Value { + val, err := zson.MarshalZNG(struct { Address zed.Value `zed:"address"` Mask zed.Value `zed:"mask"` }{address, mask}) @@ -84,10 +81,10 @@ type CIDRMatch struct { var errMatch = errors.New("match") -func (c *CIDRMatch) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (c *CIDRMatch) Call(_ zed.Allocator, args []zed.Value) zed.Value { maskVal := args[0] if maskVal.Type().ID() != zed.IDNet { - return c.zctx.WrapError(ectx.Arena(), "cidr_match: not a net", maskVal) + return c.zctx.WrapError("cidr_match: not a net", maskVal) } prefix := zed.DecodeNet(maskVal.Bytes()) err := args[1].Walk(func(typ zed.Type, body zcode.Bytes) error { diff --git a/runtime/sam/expr/function/ksuid.go b/runtime/sam/expr/function/ksuid.go index 9cca1e3bf6..1e91a144dc 100644 --- a/runtime/sam/expr/function/ksuid.go +++ b/runtime/sam/expr/function/ksuid.go @@ -2,7 +2,6 @@ package function import ( "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/segmentio/ksuid" ) @@ -11,30 +10,30 @@ type KSUIDToString struct { zctx *zed.Context } -func (k *KSUIDToString) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (k *KSUIDToString) Call(_ zed.Allocator, args []zed.Value) zed.Value { if len(args) == 0 { - return ectx.Arena().NewBytes(ksuid.New().Bytes()) + return zed.NewBytes(ksuid.New().Bytes()) } val := args[0] switch val.Type().ID() { case zed.IDBytes: if val.IsNull() { - return k.zctx.NewErrorf(ectx.Arena(), "ksuid: illegal null argument") + return k.zctx.NewErrorf("ksuid: illegal null argument") } // XXX GC id, err := ksuid.FromBytes(val.Bytes()) if err != nil { panic(err) } - return ectx.Arena().NewString(id.String()) + return zed.NewString(id.String()) case zed.IDString: // XXX GC id, err := ksuid.Parse(string(val.Bytes())) if err != nil { - return k.zctx.WrapError(ectx.Arena(), "ksuid: "+err.Error(), val) + return k.zctx.WrapError("ksuid: "+err.Error(), val) } - return ectx.Arena().NewBytes(id.Bytes()) + return zed.NewBytes(id.Bytes()) default: - return k.zctx.WrapError(ectx.Arena(), "ksuid: argument must a bytes or string type", val) + return k.zctx.WrapError("ksuid: argument must a bytes or string type", val) } } diff --git a/runtime/sam/expr/function/len.go b/runtime/sam/expr/function/len.go index f53d3e2d21..f18ded70df 100644 --- a/runtime/sam/expr/function/len.go +++ b/runtime/sam/expr/function/len.go @@ -2,7 +2,6 @@ package function import ( "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" ) // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#len @@ -10,8 +9,8 @@ type LenFn struct { zctx *zed.Context } -func (l *LenFn) Call(ectx expr.Context, args []zed.Value) zed.Value { - val := args[0].Under(ectx.Arena()) +func (l *LenFn) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() var length int switch typ := zed.TypeUnder(val.Type()).(type) { case *zed.TypeOfNull: @@ -26,15 +25,15 @@ func (l *LenFn) Call(ectx expr.Context, args []zed.Value) zed.Value { case *zed.TypeOfBytes, *zed.TypeOfString, *zed.TypeOfIP, *zed.TypeOfNet: length = len(val.Bytes()) case *zed.TypeError: - return l.zctx.WrapError(ectx.Arena(), "len()", val) + return l.zctx.WrapError("len()", val) case *zed.TypeOfType: t, err := l.zctx.LookupByValue(val.Bytes()) if err != nil { - return l.zctx.NewError(ectx.Arena(), err) + return l.zctx.NewError(err) } length = TypeLength(t) default: - return l.zctx.WrapError(ectx.Arena(), "len: bad type", val) + return l.zctx.WrapError("len: bad type", val) } return zed.NewInt64(int64(length)) } diff --git a/runtime/sam/expr/function/math.go b/runtime/sam/expr/function/math.go index a1e1937a56..5bbcd62c87 100644 --- a/runtime/sam/expr/function/math.go +++ b/runtime/sam/expr/function/math.go @@ -5,7 +5,6 @@ import ( "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/anymath" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/runtime/sam/expr/coerce" ) @@ -14,7 +13,7 @@ type Abs struct { zctx *zed.Context } -func (a *Abs) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (a *Abs) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] switch id := val.Type().ID(); { case zed.IsUnsigned(id): @@ -28,7 +27,7 @@ func (a *Abs) Call(ectx expr.Context, args []zed.Value) zed.Value { case zed.IsFloat(id): return zed.NewFloat(val.Type(), math.Abs(val.Float())) } - return a.zctx.WrapError(ectx.Arena(), "abs: not a number", val) + return a.zctx.WrapError("abs: not a number", val) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#ceil @@ -36,7 +35,7 @@ type Ceil struct { zctx *zed.Context } -func (c *Ceil) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (c *Ceil) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] switch id := val.Type().ID(); { case zed.IsUnsigned(id) || zed.IsSigned(id): @@ -44,7 +43,7 @@ func (c *Ceil) Call(ectx expr.Context, args []zed.Value) zed.Value { case zed.IsFloat(id): return zed.NewFloat(val.Type(), math.Ceil(val.Float())) } - return c.zctx.WrapError(ectx.Arena(), "ceil: not a number", val) + return c.zctx.WrapError("ceil: not a number", val) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#floor @@ -52,7 +51,7 @@ type Floor struct { zctx *zed.Context } -func (f *Floor) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (f *Floor) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] switch id := val.Type().ID(); { case zed.IsUnsigned(id) || zed.IsSigned(id): @@ -60,7 +59,7 @@ func (f *Floor) Call(ectx expr.Context, args []zed.Value) zed.Value { case zed.IsFloat(id): return zed.NewFloat(val.Type(), math.Floor(val.Float())) } - return f.zctx.WrapError(ectx.Arena(), "floor: not a number", val) + return f.zctx.WrapError("floor: not a number", val) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#log @@ -68,13 +67,13 @@ type Log struct { zctx *zed.Context } -func (l *Log) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (l *Log) Call(_ zed.Allocator, args []zed.Value) zed.Value { x, ok := coerce.ToFloat(args[0]) if !ok { - return l.zctx.WrapError(ectx.Arena(), "log: not a number", args[0]) + return l.zctx.WrapError("log: not a number", args[0]) } if x <= 0 { - return l.zctx.WrapError(ectx.Arena(), "log: illegal argument", args[0]) + return l.zctx.WrapError("log: illegal argument", args[0]) } return zed.NewFloat64(math.Log(x)) } @@ -85,7 +84,7 @@ type reducer struct { fn *anymath.Function } -func (r *reducer) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (r *reducer) Call(_ zed.Allocator, args []zed.Value) zed.Value { val0 := args[0] switch id := val0.Type().ID(); { case zed.IsUnsigned(id): @@ -93,7 +92,7 @@ func (r *reducer) Call(ectx expr.Context, args []zed.Value) zed.Value { for _, val := range args[1:] { v, ok := coerce.ToUint(val) if !ok { - return r.zctx.WrapError(ectx.Arena(), r.name+": not a number", val) + return r.zctx.WrapError(r.name+": not a number", val) } result = r.fn.Uint64(result, v) } @@ -105,7 +104,7 @@ func (r *reducer) Call(ectx expr.Context, args []zed.Value) zed.Value { // floats to ints if we hit a float first v, ok := coerce.ToInt(val) if !ok { - return r.zctx.WrapError(ectx.Arena(), r.name+": not a number", val) + return r.zctx.WrapError(r.name+": not a number", val) } result = r.fn.Int64(result, v) } @@ -117,13 +116,13 @@ func (r *reducer) Call(ectx expr.Context, args []zed.Value) zed.Value { for _, val := range args[1:] { v, ok := coerce.ToFloat(val) if !ok { - return r.zctx.WrapError(ectx.Arena(), r.name+": not a number", val) + return r.zctx.WrapError(r.name+": not a number", val) } result = r.fn.Float64(result, v) } return zed.NewFloat64(result) } - return r.zctx.WrapError(ectx.Arena(), r.name+": not a number", val0) + return r.zctx.WrapError(r.name+": not a number", val0) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#round @@ -131,7 +130,7 @@ type Round struct { zctx *zed.Context } -func (r *Round) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (r *Round) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] switch id := val.Type().ID(); { case zed.IsUnsigned(id) || zed.IsSigned(id): @@ -139,7 +138,7 @@ func (r *Round) Call(ectx expr.Context, args []zed.Value) zed.Value { case zed.IsFloat(id): return zed.NewFloat(val.Type(), math.Round(val.Float())) } - return r.zctx.WrapError(ectx.Arena(), "round: not a number", val) + return r.zctx.WrapError("round: not a number", val) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#pow @@ -147,14 +146,14 @@ type Pow struct { zctx *zed.Context } -func (p *Pow) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (p *Pow) Call(_ zed.Allocator, args []zed.Value) zed.Value { x, ok := coerce.ToFloat(args[0]) if !ok { - return p.zctx.WrapError(ectx.Arena(), "pow: not a number", args[0]) + return p.zctx.WrapError("pow: not a number", args[0]) } y, ok := coerce.ToFloat(args[1]) if !ok { - return p.zctx.WrapError(ectx.Arena(), "pow: not a number", args[1]) + return p.zctx.WrapError("pow: not a number", args[1]) } return zed.NewFloat64(math.Pow(x, y)) } @@ -164,10 +163,10 @@ type Sqrt struct { zctx *zed.Context } -func (s *Sqrt) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (s *Sqrt) Call(_ zed.Allocator, args []zed.Value) zed.Value { x, ok := coerce.ToFloat(args[0]) if !ok { - return s.zctx.WrapError(ectx.Arena(), "sqrt: not a number", args[0]) + return s.zctx.WrapError("sqrt: not a number", args[0]) } return zed.NewFloat64(math.Sqrt(x)) } diff --git a/runtime/sam/expr/function/nestdotted.go b/runtime/sam/expr/function/nestdotted.go index 591ac05f16..23f06d9668 100644 --- a/runtime/sam/expr/function/nestdotted.go +++ b/runtime/sam/expr/function/nestdotted.go @@ -3,7 +3,6 @@ package function import ( "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/field" - "github.com/brimdata/zed/runtime/sam/expr" ) // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#nest_dotted.md @@ -56,11 +55,11 @@ func (n *NestDotted) lookupBuilderAndType(in *zed.TypeRecord) (*zed.RecordBuilde return b, typ, nil } -func (n *NestDotted) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (n *NestDotted) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[len(args)-1] b, typ, err := n.lookupBuilderAndType(zed.TypeRecordOf(val.Type())) if err != nil { - return n.zctx.WrapError(ectx.Arena(), "nest_dotted(): "+err.Error(), val) + return n.zctx.WrapError("nest_dotted(): "+err.Error(), val) } if b == nil { return val @@ -73,5 +72,5 @@ func (n *NestDotted) Call(ectx expr.Context, args []zed.Value) zed.Value { if err != nil { panic(err) } - return ectx.Arena().New(typ, zbytes) + return zed.NewValue(typ, zbytes) } diff --git a/runtime/sam/expr/function/parse.go b/runtime/sam/expr/function/parse.go index 64698237e6..0ab6d75067 100644 --- a/runtime/sam/expr/function/parse.go +++ b/runtime/sam/expr/function/parse.go @@ -3,9 +3,10 @@ package function import ( "net/url" "strconv" + "strings" "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" + "github.com/brimdata/zed/zio/zsonio" "github.com/brimdata/zed/zson" ) @@ -15,15 +16,15 @@ type ParseURI struct { marshaler *zson.MarshalZNGContext } -func (p *ParseURI) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (p *ParseURI) Call(_ zed.Allocator, args []zed.Value) zed.Value { in := args[0] if !in.IsString() || in.IsNull() { - return p.zctx.WrapError(ectx.Arena(), "parse_uri: non-empty string arg required", in) + return p.zctx.WrapError("parse_uri: non-empty string arg required", in) } s := zed.DecodeString(in.Bytes()) u, err := url.Parse(s) if err != nil { - return p.zctx.WrapError(ectx.Arena(), "parse_uri: "+err.Error(), in) + return p.zctx.WrapError("parse_uri: "+err.Error(), in) } var v struct { Scheme *string `zed:"scheme"` @@ -54,7 +55,7 @@ func (p *ParseURI) Call(ectx expr.Context, args []zed.Value) zed.Value { if portString := u.Port(); portString != "" { u64, err := strconv.ParseUint(portString, 10, 16) if err != nil { - return p.zctx.WrapError(ectx.Arena(), "parse_uri: invalid port: "+portString, in) + return p.zctx.WrapError("parse_uri: invalid port: "+portString, in) } u16 := uint16(u64) v.Port = &u16 @@ -68,7 +69,7 @@ func (p *ParseURI) Call(ectx expr.Context, args []zed.Value) zed.Value { if u.Fragment != "" { v.Fragment = &u.Fragment } - out, err := p.marshaler.Marshal(ectx.Arena(), v) + out, err := p.marshaler.Marshal(v) if err != nil { panic(err) } @@ -78,23 +79,30 @@ func (p *ParseURI) Call(ectx expr.Context, args []zed.Value) zed.Value { // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#parse_zson type ParseZSON struct { zctx *zed.Context + sr *strings.Reader + zr *zsonio.Reader } func newParseZSON(zctx *zed.Context) *ParseZSON { - return &ParseZSON{zctx} + var sr strings.Reader + return &ParseZSON{zctx, &sr, zsonio.NewReader(zctx, &sr)} } -func (p *ParseZSON) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (p *ParseZSON) Call(_ zed.Allocator, args []zed.Value) zed.Value { in := args[0] if !in.IsString() { - return p.zctx.WrapError(ectx.Arena(), "parse_zson: string arg required", in) + return p.zctx.WrapError("parse_zson: string arg required", in) } if in.IsNull() { return zed.Null } - val, err := zson.ParseValue(p.zctx, ectx.Arena(), zed.DecodeString(in.Bytes())) + p.sr.Reset(zed.DecodeString(in.Bytes())) + val, err := p.zr.Read() if err != nil { - return p.zctx.WrapError(ectx.Arena(), "parse_zson: "+err.Error(), in) + return p.zctx.WrapError("parse_zson: "+err.Error(), in) } - return val + if val == nil { + return zed.Null + } + return *val } diff --git a/runtime/sam/expr/function/regexp.go b/runtime/sam/expr/function/regexp.go index 8e396a277a..3a117c1a9b 100644 --- a/runtime/sam/expr/function/regexp.go +++ b/runtime/sam/expr/function/regexp.go @@ -4,7 +4,6 @@ import ( "regexp" "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" ) @@ -18,9 +17,9 @@ type Regexp struct { zctx *zed.Context } -func (r *Regexp) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (r *Regexp) Call(_ zed.Allocator, args []zed.Value) zed.Value { if !args[0].IsString() { - return r.zctx.WrapError(ectx.Arena(), "regexp: string required for first arg", args[0]) + return r.zctx.WrapError("regexp: string required for first arg", args[0]) } s := zed.DecodeString(args[0].Bytes()) if r.restr != s { @@ -28,10 +27,10 @@ func (r *Regexp) Call(ectx expr.Context, args []zed.Value) zed.Value { r.re, r.err = regexp.Compile(r.restr) } if r.err != nil { - return r.zctx.NewErrorf(ectx.Arena(), "regexp: %s", r.err) + return r.zctx.NewErrorf("regexp: %s", r.err) } if !args[1].IsString() { - return r.zctx.WrapError(ectx.Arena(), "regexp: string required for second arg", args[1]) + return r.zctx.WrapError("regexp: string required for second arg", args[1]) } r.builder.Reset() for _, b := range r.re.FindSubmatch(args[1].Bytes()) { @@ -40,7 +39,7 @@ func (r *Regexp) Call(ectx expr.Context, args []zed.Value) zed.Value { if r.typ == nil { r.typ = r.zctx.LookupTypeArray(zed.TypeString) } - return ectx.Arena().New(r.typ, r.builder.Bytes()) + return zed.NewValue(r.typ, r.builder.Bytes()) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#regexp_replace @@ -51,27 +50,27 @@ type RegexpReplace struct { err error } -func (r *RegexpReplace) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (r *RegexpReplace) Call(_ zed.Allocator, args []zed.Value) zed.Value { sVal := args[0] reVal := args[1] newVal := args[2] for i := range args { if !args[i].IsString() { - return r.zctx.WrapError(ectx.Arena(), "regexp_replace: string arg required", args[i]) + return r.zctx.WrapError("regexp_replace: string arg required", args[i]) } } if sVal.IsNull() { return zed.Null } if reVal.IsNull() || newVal.IsNull() { - return r.zctx.NewErrorf(ectx.Arena(), "regexp_replace: 2nd and 3rd args cannot be null") + return r.zctx.NewErrorf("regexp_replace: 2nd and 3rd args cannot be null") } if re := zed.DecodeString(reVal.Bytes()); r.restr != re { r.restr = re r.re, r.err = regexp.Compile(re) } if r.err != nil { - return r.zctx.NewErrorf(ectx.Arena(), "regexp_replace: %s", r.err) + return r.zctx.NewErrorf("regexp_replace: %s", r.err) } - return ectx.Arena().NewString(string(r.re.ReplaceAll(sVal.Bytes(), newVal.Bytes()))) + return zed.NewString(string(r.re.ReplaceAll(sVal.Bytes(), newVal.Bytes()))) } diff --git a/runtime/sam/expr/function/string.go b/runtime/sam/expr/function/string.go index f40d50ccd3..f33d879522 100644 --- a/runtime/sam/expr/function/string.go +++ b/runtime/sam/expr/function/string.go @@ -6,7 +6,6 @@ import ( "github.com/agnivade/levenshtein" "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" ) @@ -15,26 +14,26 @@ type Replace struct { zctx *zed.Context } -func (r *Replace) Call(ectx expr.Context, args []zed.Value) zed.Value { - args = underAll(ectx.Arena(), args) +func (r *Replace) Call(_ zed.Allocator, args []zed.Value) zed.Value { + args = underAll(args) sVal := args[0] oldVal := args[1] newVal := args[2] for i := range args { if !args[i].IsString() { - return r.zctx.WrapError(ectx.Arena(), "replace: string arg required", args[i]) + return r.zctx.WrapError("replace: string arg required", args[i]) } } if sVal.IsNull() { return zed.Null } if oldVal.IsNull() || newVal.IsNull() { - return r.zctx.NewErrorf(ectx.Arena(), "replace: an input arg is null") + return r.zctx.NewErrorf("replace: an input arg is null") } s := zed.DecodeString(sVal.Bytes()) old := zed.DecodeString(oldVal.Bytes()) new := zed.DecodeString(newVal.Bytes()) - return ectx.Arena().NewString(strings.ReplaceAll(s, old, new)) + return zed.NewString(strings.ReplaceAll(s, old, new)) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#run_len @@ -42,10 +41,10 @@ type RuneLen struct { zctx *zed.Context } -func (r *RuneLen) Call(ectx expr.Context, args []zed.Value) zed.Value { - val := args[0].Under(ectx.Arena()) +func (r *RuneLen) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() if !val.IsString() { - return r.zctx.WrapError(ectx.Arena(), "rune_len: string arg required", val) + return r.zctx.WrapError("rune_len: string arg required", val) } if val.IsNull() { return zed.NewInt64(0) @@ -59,16 +58,16 @@ type ToLower struct { zctx *zed.Context } -func (t *ToLower) Call(ectx expr.Context, args []zed.Value) zed.Value { - val := args[0].Under(ectx.Arena()) +func (t *ToLower) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() if !val.IsString() { - return t.zctx.WrapError(ectx.Arena(), "lower: string arg required", val) + return t.zctx.WrapError("lower: string arg required", val) } if val.IsNull() { return zed.NullString } s := zed.DecodeString(val.Bytes()) - return ectx.Arena().NewString(strings.ToLower(s)) + return zed.NewString(strings.ToLower(s)) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#upper @@ -76,16 +75,16 @@ type ToUpper struct { zctx *zed.Context } -func (t *ToUpper) Call(ectx expr.Context, args []zed.Value) zed.Value { - val := args[0].Under(ectx.Arena()) +func (t *ToUpper) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() if !val.IsString() { - return t.zctx.WrapError(ectx.Arena(), "upper: string arg required", val) + return t.zctx.WrapError("upper: string arg required", val) } if val.IsNull() { return zed.NullString } s := zed.DecodeString(val.Bytes()) - return ectx.Arena().NewString(strings.ToUpper(s)) + return zed.NewString(strings.ToUpper(s)) } type Trim struct { @@ -93,16 +92,16 @@ type Trim struct { } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#trim -func (t *Trim) Call(ectx expr.Context, args []zed.Value) zed.Value { - val := args[0].Under(ectx.Arena()) +func (t *Trim) Call(_ zed.Allocator, args []zed.Value) zed.Value { + val := args[0].Under() if !val.IsString() { - return t.zctx.WrapError(ectx.Arena(), "trim: string arg required", val) + return t.zctx.WrapError("trim: string arg required", val) } if val.IsNull() { return zed.NullString } s := zed.DecodeString(val.Bytes()) - return ectx.Arena().NewString(strings.TrimSpace(s)) + return zed.NewString(strings.TrimSpace(s)) } // // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#split @@ -118,16 +117,16 @@ func newSplit(zctx *zed.Context) *Split { } } -func (s *Split) Call(ectx expr.Context, args []zed.Value) zed.Value { - args = underAll(ectx.Arena(), args) +func (s *Split) Call(_ zed.Allocator, args []zed.Value) zed.Value { + args = underAll(args) for i := range args { if !args[i].IsString() { - return s.zctx.WrapError(ectx.Arena(), "split: string arg required", args[i]) + return s.zctx.WrapError("split: string arg required", args[i]) } } sVal, sepVal := args[0], args[1] if sVal.IsNull() || sepVal.IsNull() { - return ectx.Arena().New(s.typ, nil) + return zed.NewValue(s.typ, nil) } str := zed.DecodeString(sVal.Bytes()) sep := zed.DecodeString(sepVal.Bytes()) @@ -136,7 +135,7 @@ func (s *Split) Call(ectx expr.Context, args []zed.Value) zed.Value { for _, substr := range splits { b = zcode.Append(b, zed.EncodeString(substr)) } - return ectx.Arena().New(s.typ, b) + return zed.NewValue(s.typ, b) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#join @@ -145,18 +144,18 @@ type Join struct { builder strings.Builder } -func (j *Join) Call(ectx expr.Context, args []zed.Value) zed.Value { - args = underAll(ectx.Arena(), args) +func (j *Join) Call(_ zed.Allocator, args []zed.Value) zed.Value { + args = underAll(args) splitsVal := args[0] typ, ok := zed.TypeUnder(splitsVal.Type()).(*zed.TypeArray) if !ok || typ.Type.ID() != zed.IDString { - return j.zctx.WrapError(ectx.Arena(), "join: array of string arg required", splitsVal) + return j.zctx.WrapError("join: array of string arg required", splitsVal) } var separator string if len(args) == 2 { sepVal := args[1] if !sepVal.IsString() { - return j.zctx.WrapError(ectx.Arena(), "join: separator must be string", sepVal) + return j.zctx.WrapError("join: separator must be string", sepVal) } separator = zed.DecodeString(sepVal.Bytes()) } @@ -169,7 +168,7 @@ func (j *Join) Call(ectx expr.Context, args []zed.Value) zed.Value { b.WriteString(zed.DecodeString(it.Next())) sep = separator } - return ectx.Arena().NewString(b.String()) + return zed.NewString(b.String()) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#levenshtein @@ -177,14 +176,14 @@ type Levenshtein struct { zctx *zed.Context } -func (l *Levenshtein) Call(ectx expr.Context, args []zed.Value) zed.Value { - args = underAll(ectx.Arena(), args) +func (l *Levenshtein) Call(_ zed.Allocator, args []zed.Value) zed.Value { + args = underAll(args) a, b := args[0], args[1] if !a.IsString() { - return l.zctx.WrapError(ectx.Arena(), "levenshtein: string args required", a) + return l.zctx.WrapError("levenshtein: string args required", a) } if !b.IsString() { - return l.zctx.WrapError(ectx.Arena(), "levenshtein: string args required", b) + return l.zctx.WrapError("levenshtein: string args required", b) } as, bs := zed.DecodeString(a.Bytes()), zed.DecodeString(b.Bytes()) return zed.NewInt64(int64(levenshtein.ComputeDistance(as, bs))) diff --git a/runtime/sam/expr/function/time.go b/runtime/sam/expr/function/time.go index 049d8d2476..112ba95ee5 100644 --- a/runtime/sam/expr/function/time.go +++ b/runtime/sam/expr/function/time.go @@ -3,7 +3,6 @@ package function import ( "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/nano" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/runtime/sam/expr/coerce" "github.com/lestrrat-go/strftime" ) @@ -11,7 +10,7 @@ import ( // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#now type Now struct{} -func (n *Now) Call(_ expr.Context, _ []zed.Value) zed.Value { +func (n *Now) Call(_ zed.Allocator, _ []zed.Value) zed.Value { return zed.NewTime(nano.Now()) } @@ -21,7 +20,7 @@ type Bucket struct { zctx *zed.Context } -func (b *Bucket) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (b *Bucket) Call(_ zed.Allocator, args []zed.Value) zed.Value { tsArg := args[0] binArg := args[1] if tsArg.IsNull() || binArg.IsNull() { @@ -33,7 +32,7 @@ func (b *Bucket) Call(ectx expr.Context, args []zed.Value) zed.Value { } else { d, ok := coerce.ToInt(binArg) if !ok { - return b.zctx.WrapError(ectx.Arena(), b.name+": second argument is not a duration or number", binArg) + return b.zctx.WrapError(b.name+": second argument is not a duration or number", binArg) } bin = nano.Duration(d) * nano.Second } @@ -43,7 +42,7 @@ func (b *Bucket) Call(ectx expr.Context, args []zed.Value) zed.Value { } v, ok := coerce.ToInt(tsArg) if !ok { - return b.zctx.WrapError(ectx.Arena(), b.name+": first argument is not a time", tsArg) + return b.zctx.WrapError(b.name+": first argument is not a time", tsArg) } return zed.NewTime(nano.Ts(v).Trunc(bin)) } @@ -54,21 +53,21 @@ type Strftime struct { formatter *strftime.Strftime } -func (s *Strftime) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (s *Strftime) Call(_ zed.Allocator, args []zed.Value) zed.Value { formatArg, timeArg := args[0], args[1] if !formatArg.IsString() { - return s.zctx.WrapError(ectx.Arena(), "strftime: string value required for format arg", formatArg) + return s.zctx.WrapError("strftime: string value required for format arg", formatArg) } if zed.TypeUnder(timeArg.Type()) != zed.TypeTime { - return s.zctx.WrapError(ectx.Arena(), "strftime: time value required for time arg", args[1]) + return s.zctx.WrapError("strftime: time value required for time arg", args[1]) } format := formatArg.AsString() if s.formatter == nil || s.formatter.Pattern() != format { var err error if s.formatter, err = strftime.New(format); err != nil { - return s.zctx.WrapError(ectx.Arena(), "strftime: "+err.Error(), formatArg) + return s.zctx.WrapError("strftime: "+err.Error(), formatArg) } } out := s.formatter.FormatString(timeArg.AsTime().Time()) - return ectx.Arena().NewString(out) + return zed.NewString(out) } diff --git a/runtime/sam/expr/function/types.go b/runtime/sam/expr/function/types.go index 5b56486825..ccf4b7e597 100644 --- a/runtime/sam/expr/function/types.go +++ b/runtime/sam/expr/function/types.go @@ -2,7 +2,6 @@ package function import ( "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" "github.com/brimdata/zed/zson" ) @@ -12,17 +11,17 @@ type TypeOf struct { zctx *zed.Context } -func (t *TypeOf) Call(ectx expr.Context, args []zed.Value) zed.Value { - return t.zctx.LookupTypeValue(ectx.Arena(), args[0].Type()) +func (t *TypeOf) Call(_ zed.Allocator, args []zed.Value) zed.Value { + return t.zctx.LookupTypeValue(args[0].Type()) } type typeUnder struct { zctx *zed.Context } -func (t *typeUnder) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (t *typeUnder) Call(_ zed.Allocator, args []zed.Value) zed.Value { typ := zed.TypeUnder(args[0].Type()) - return t.zctx.LookupTypeValue(ectx.Arena(), typ) + return t.zctx.LookupTypeValue(typ) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#nameof @@ -30,10 +29,10 @@ type NameOf struct { zctx *zed.Context } -func (n *NameOf) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (n *NameOf) Call(_ zed.Allocator, args []zed.Value) zed.Value { typ := args[0].Type() if named, ok := typ.(*zed.TypeNamed); ok { - return ectx.Arena().NewString(named.Name) + return zed.NewString(named.Name) } if typ.ID() == zed.IDType { var err error @@ -41,10 +40,10 @@ func (n *NameOf) Call(ectx expr.Context, args []zed.Value) zed.Value { panic(err) } if named, ok := typ.(*zed.TypeNamed); ok { - return ectx.Arena().NewString(named.Name) + return zed.NewString(named.Name) } } - return n.zctx.Missing(ectx.Arena()) + return n.zctx.Missing() } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#typename @@ -52,26 +51,26 @@ type typeName struct { zctx *zed.Context } -func (t *typeName) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (t *typeName) Call(_ zed.Allocator, args []zed.Value) zed.Value { if zed.TypeUnder(args[0].Type()) != zed.TypeString { - return t.zctx.WrapError(ectx.Arena(), "typename: first argument not a string", args[0]) + return t.zctx.WrapError("typename: first argument not a string", args[0]) } name := string(args[0].Bytes()) if len(args) == 1 { typ := t.zctx.LookupTypeDef(name) if typ == nil { - return t.zctx.Missing(ectx.Arena()) + return t.zctx.Missing() } - return t.zctx.LookupTypeValue(ectx.Arena(), typ) + return t.zctx.LookupTypeValue(typ) } if zed.TypeUnder(args[1].Type()) != zed.TypeType { - return t.zctx.WrapError(ectx.Arena(), "typename: second argument not a type value", args[1]) + return t.zctx.WrapError("typename: second argument not a type value", args[1]) } typ, err := t.zctx.LookupByValue(args[1].Bytes()) if err != nil { - return t.zctx.NewError(ectx.Arena(), err) + return t.zctx.NewError(err) } - return t.zctx.LookupTypeValue(ectx.Arena(), typ) + return t.zctx.LookupTypeValue(typ) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#error @@ -79,14 +78,14 @@ type Error struct { zctx *zed.Context } -func (e *Error) Call(ectx expr.Context, args []zed.Value) zed.Value { - return ectx.Arena().New(e.zctx.LookupTypeError(args[0].Type()), args[0].Bytes()) +func (e *Error) Call(_ zed.Allocator, args []zed.Value) zed.Value { + return zed.NewValue(e.zctx.LookupTypeError(args[0].Type()), args[0].Bytes()) } // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#iserr type IsErr struct{} -func (*IsErr) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (*IsErr) Call(_ zed.Allocator, args []zed.Value) zed.Value { return zed.NewBool(args[0].IsError()) } @@ -95,7 +94,7 @@ type Is struct { zctx *zed.Context } -func (i *Is) Call(_ expr.Context, args []zed.Value) zed.Value { +func (i *Is) Call(_ zed.Allocator, args []zed.Value) zed.Value { zvSubject := args[0] zvTypeVal := args[1] if len(args) == 3 { @@ -122,7 +121,7 @@ func NewHasError() *HasError { } } -func (h *HasError) Call(_ expr.Context, args []zed.Value) zed.Value { +func (h *HasError) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] hasError, _ := h.hasError(val.Type(), val.Bytes()) return zed.NewBool(hasError) @@ -191,10 +190,10 @@ type Quiet struct { zctx *zed.Context } -func (q *Quiet) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (q *Quiet) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] if val.IsMissing() { - return q.zctx.Quiet(ectx.Arena()) + return q.zctx.Quiet() } return val } @@ -204,7 +203,7 @@ type Kind struct { zctx *zed.Context } -func (k *Kind) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (k *Kind) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] var typ zed.Type if _, ok := zed.TypeUnder(val.Type()).(*zed.TypeOfType); ok { @@ -216,5 +215,5 @@ func (k *Kind) Call(ectx expr.Context, args []zed.Value) zed.Value { } else { typ = val.Type() } - return ectx.Arena().NewString(typ.Kind().String()) + return zed.NewString(typ.Kind().String()) } diff --git a/runtime/sam/expr/function/under.go b/runtime/sam/expr/function/under.go index f03a190bbf..28920cb92f 100644 --- a/runtime/sam/expr/function/under.go +++ b/runtime/sam/expr/function/under.go @@ -2,7 +2,6 @@ package function import ( "github.com/brimdata/zed" - "github.com/brimdata/zed/runtime/sam/expr" ) // https://github.com/brimdata/zed/blob/main/docs/language/functions.md#under @@ -10,21 +9,21 @@ type Under struct { zctx *zed.Context } -func (u *Under) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (u *Under) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] switch typ := args[0].Type().(type) { case *zed.TypeNamed: - return ectx.Arena().New(typ.Type, val.Bytes()) + return zed.NewValue(typ.Type, val.Bytes()) case *zed.TypeError: - return ectx.Arena().New(typ.Type, val.Bytes()) + return zed.NewValue(typ.Type, val.Bytes()) case *zed.TypeUnion: - return ectx.Arena().New(typ.Untag(val.Bytes())) + return zed.NewValue(typ.Untag(val.Bytes())) case *zed.TypeOfType: t, err := u.zctx.LookupByValue(val.Bytes()) if err != nil { - return u.zctx.NewError(ectx.Arena(), err) + return u.zctx.NewError(err) } - return u.zctx.LookupTypeValue(ectx.Arena(), zed.TypeUnder(t)) + return u.zctx.LookupTypeValue(zed.TypeUnder(t)) default: return val } diff --git a/runtime/sam/expr/function/unflatten.go b/runtime/sam/expr/function/unflatten.go index ea0cfb8eab..ae81a348d8 100644 --- a/runtime/sam/expr/function/unflatten.go +++ b/runtime/sam/expr/function/unflatten.go @@ -5,7 +5,6 @@ import ( "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/field" - "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zcode" "github.com/brimdata/zed/zson" ) @@ -29,7 +28,7 @@ func NewUnflatten(zctx *zed.Context) *Unflatten { } } -func (u *Unflatten) Call(ectx expr.Context, args []zed.Value) zed.Value { +func (u *Unflatten) Call(_ zed.Allocator, args []zed.Value) zed.Value { val := args[0] array, ok := zed.TypeUnder(val.Type()).(*zed.TypeArray) if !ok { @@ -43,8 +42,7 @@ func (u *Unflatten) Call(ectx expr.Context, args []zed.Value) zed.Value { bytes := it.Next() path, typ, vb, err := u.parseElem(array.Type, bytes) if err != nil { - arena := ectx.Arena() - return u.zctx.WrapError(arena, err.Error(), arena.New(array.Type, bytes)) + return u.zctx.WrapError(err.Error(), zed.NewValue(array.Type, bytes)) } if typ == nil { continue @@ -64,9 +62,9 @@ func (u *Unflatten) Call(ectx expr.Context, args []zed.Value) zed.Value { return typ, value }) if err != nil { - return u.zctx.WrapError(ectx.Arena(), err.Error(), val) + return u.zctx.WrapError(err.Error(), val) } - return ectx.Arena().New(typ, u.builder.Bytes()) + return zed.NewValue(typ, u.builder.Bytes()) } func (u *Unflatten) parseElem(inner zed.Type, vb zcode.Bytes) (field.Path, zed.Type, zcode.Bytes, error) { diff --git a/runtime/sam/expr/literal.go b/runtime/sam/expr/literal.go index b997b5f172..e9326557a5 100644 --- a/runtime/sam/expr/literal.go +++ b/runtime/sam/expr/literal.go @@ -9,7 +9,7 @@ type Literal struct { var _ Evaluator = (*Literal)(nil) func NewLiteral(val zed.Value) *Literal { - return &Literal{val} + return &Literal{val: val} } func (l Literal) Eval(Context, zed.Value) zed.Value { diff --git a/runtime/sam/expr/lval.go b/runtime/sam/expr/lval.go index 5502a5ba72..e1f60d3080 100644 --- a/runtime/sam/expr/lval.go +++ b/runtime/sam/expr/lval.go @@ -82,7 +82,7 @@ func (l *ExprLvalElem) Eval(ectx Context, this zed.Value) (string, error) { } func lvalErr(ectx Context, errVal zed.Value) error { - val := ectx.Arena().New(errVal.Type().(*zed.TypeError).Type, errVal.Bytes()) + val := zed.NewValue(errVal.Type().(*zed.TypeError).Type, errVal.Bytes()) if val.IsString() { return errors.New(val.AsString()) } diff --git a/runtime/sam/expr/map.go b/runtime/sam/expr/map.go index 2068f5da90..4d15b557d3 100644 --- a/runtime/sam/expr/map.go +++ b/runtime/sam/expr/map.go @@ -26,9 +26,9 @@ func (a *mapCall) Eval(ectx Context, in zed.Value) zed.Value { if val.IsError() { return val } - elems, err := val.Elements(ectx.Arena()) + elems, err := val.Elements() if err != nil { - return a.zctx.WrapError(ectx.Arena(), err.Error(), in) + return a.zctx.WrapError(err.Error(), in) } if len(elems) == 0 { return val @@ -43,9 +43,9 @@ func (a *mapCall) Eval(ectx Context, in zed.Value) zed.Value { inner := a.innerType(a.types) bytes := a.buildVal(inner, a.vals) if _, ok := zed.TypeUnder(val.Type()).(*zed.TypeSet); ok { - return ectx.Arena().New(a.zctx.LookupTypeSet(inner), zed.NormalizeSet(bytes)) + return zed.NewValue(a.zctx.LookupTypeSet(inner), zed.NormalizeSet(bytes)) } - return ectx.Arena().New(a.zctx.LookupTypeArray(inner), bytes) + return zed.NewValue(a.zctx.LookupTypeArray(inner), bytes) } func (a *mapCall) buildVal(inner zed.Type, vals []zed.Value) []byte { diff --git a/runtime/sam/expr/putter.go b/runtime/sam/expr/putter.go index 3844c1a778..3b3dc5250e 100644 --- a/runtime/sam/expr/putter.go +++ b/runtime/sam/expr/putter.go @@ -318,19 +318,19 @@ func (p *Putter) Eval(ectx Context, this zed.Value) zed.Value { // propagate errors return this } - return p.zctx.WrapError(ectx.Arena(), "put: not a record", this) + return p.zctx.WrapError("put: not a record", this) } vals, paths, err := p.eval(ectx, this) if err != nil { - return p.zctx.WrapError(ectx.Arena(), fmt.Sprintf("put: %s", err), this) + return p.zctx.WrapError(fmt.Sprintf("put: %s", err), this) } if len(vals) == 0 { return this } rule, err := p.lookupRule(recType, vals, paths) if err != nil { - return p.zctx.WrapError(ectx.Arena(), err.Error(), this) + return p.zctx.WrapError(err.Error(), this) } bytes := rule.step.build(this.Bytes(), &p.builder, vals) - return ectx.Arena().New(rule.typ, bytes) + return zed.NewValue(rule.typ, bytes) } diff --git a/runtime/sam/expr/renamer.go b/runtime/sam/expr/renamer.go index 0956ca1a60..8ba57ab704 100644 --- a/runtime/sam/expr/renamer.go +++ b/runtime/sam/expr/renamer.go @@ -34,7 +34,7 @@ func (r *Renamer) Eval(ectx Context, this zed.Value) zed.Value { } srcs, dsts, err := r.evalFields(ectx, this) if err != nil { - return r.zctx.WrapError(ectx.Arena(), fmt.Sprintf("rename: %s", err), this) + return r.zctx.WrapError(fmt.Sprintf("rename: %s", err), this) } id := this.Type().ID() m, ok := r.typeMap[id] @@ -48,11 +48,11 @@ func (r *Renamer) Eval(ectx Context, this zed.Value) zed.Value { var err error typ, err = r.computeType(zed.TypeRecordOf(this.Type()), srcs, dsts) if err != nil { - return r.zctx.WrapError(ectx.Arena(), fmt.Sprintf("rename: %s", err), this) + return r.zctx.WrapError(fmt.Sprintf("rename: %s", err), this) } m[string(r.fieldsStr)] = typ } - return ectx.Arena().New(typ, this.Bytes()) + return zed.NewValue(typ, this.Bytes()) } func CheckRenameField(src, dst field.Path) error { diff --git a/runtime/sam/expr/shaper.go b/runtime/sam/expr/shaper.go index e944959f98..e521ec5c0f 100644 --- a/runtime/sam/expr/shaper.go +++ b/runtime/sam/expr/shaper.go @@ -88,7 +88,7 @@ func (s *Shaper) Eval(ectx Context, this zed.Value) zed.Value { case id == zed.IDType: typ, err := s.zctx.LookupByValue(typeVal.Bytes()) if err != nil { - return s.zctx.NewError(ectx.Arena(), err) + return s.zctx.NewError(err) } shaper, ok := s.shapers[typ] if !ok { @@ -100,7 +100,7 @@ func (s *Shaper) Eval(ectx Context, this zed.Value) zed.Value { name := zed.DecodeString(typeVal.Bytes()) return (&casterNamedType{s.zctx, s.expr, name}).Eval(ectx, this) } - return s.zctx.WrapError(ectx.Arena(), "shaper type argument is not a type", typeVal) + return s.zctx.WrapError("shaper type argument is not a type", typeVal) } type ConstShaper struct { @@ -139,18 +139,18 @@ func (c *ConstShaper) Eval(ectx Context, this zed.Value) zed.Value { } if val.IsNull() { // Null values can be shaped to any type. - return ectx.Arena().New(c.shapeTo, nil) + return zed.NewValue(c.shapeTo, nil) } id, shapeToID := val.Type().ID(), c.shapeTo.ID() if id == shapeToID { // Same underlying types but one or both are named. - return ectx.Arena().New(c.shapeTo, val.Bytes()) + return zed.NewValue(c.shapeTo, val.Bytes()) } if c.caster != nil && !zed.IsUnionType(val.Type()) { val = c.caster.Eval(ectx, val) if val.Type() != c.shapeTo && val.Type().ID() == shapeToID { // Same underlying types but one or both are named. - return ectx.Arena().New(c.shapeTo, val.Bytes()) + return zed.NewValue(c.shapeTo, val.Bytes()) } return val } @@ -159,13 +159,13 @@ func (c *ConstShaper) Eval(ectx Context, this zed.Value) zed.Value { var err error s, err = newShaper(c.zctx, c.transforms, val.Type(), c.shapeTo) if err != nil { - return c.zctx.NewError(ectx.Arena(), err) + return c.zctx.NewError(err) } c.shapers[id] = s } c.b.Reset() typ := s.step.build(c.zctx, ectx, val.Bytes(), &c.b) - return ectx.Arena().New(typ, c.b.Bytes().Body()) + return zed.NewValue(typ, c.b.Bytes().Body()) } // A shaper is a per-input type ID "spec" that contains the output @@ -446,7 +446,7 @@ func (s *step) build(zctx *zed.Context, ectx Context, in zcode.Bytes, b *zcode.B case castPrimitive: // For a successful cast, v.Type == zed.TypeUnder(s.toType). // For a failed cast, v.Type is a zed.TypeError. - v := s.caster.Eval(ectx, ectx.Arena().New(s.fromType, in)) + v := s.caster.Eval(ectx, zed.NewValue(s.fromType, in)) b.Append(v.Bytes()) if zed.TypeUnder(v.Type()) == zed.TypeUnder(s.toType) { // Prefer s.toType in case it's a named type. diff --git a/runtime/sam/expr/slice.go b/runtime/sam/expr/slice.go index 026aab1221..4f5af5e329 100644 --- a/runtime/sam/expr/slice.go +++ b/runtime/sam/expr/slice.go @@ -46,19 +46,19 @@ func (s *Slice) Eval(ectx Context, this zed.Value) zed.Value { } length = n default: - return s.zctx.WrapError(ectx.Arena(), "sliced value is not array, set, bytes, or string", elem) + return s.zctx.WrapError("sliced value is not array, set, bytes, or string", elem) } if elem.IsNull() { return elem } from, err := sliceIndex(ectx, this, s.from, length) if err != nil && err != ErrSliceIndexEmpty { - return s.zctx.NewError(ectx.Arena(), err) + return s.zctx.NewError(err) } to, err := sliceIndex(ectx, this, s.to, length) if err != nil { if err != ErrSliceIndexEmpty { - return s.zctx.NewError(ectx.Arena(), err) + return s.zctx.NewError(err) } to = length } @@ -81,7 +81,7 @@ func (s *Slice) Eval(ectx Context, this zed.Value) zed.Value { default: panic(elem.Type()) } - return ectx.Arena().New(elem.Type(), bytes) + return zed.NewValue(elem.Type(), bytes) } func sliceIndex(ectx Context, this zed.Value, slot Evaluator, length int) (int, error) { diff --git a/runtime/sam/expr/sort.go b/runtime/sam/expr/sort.go index 728de0da56..d17d6cf64f 100644 --- a/runtime/sam/expr/sort.go +++ b/runtime/sam/expr/sort.go @@ -33,9 +33,7 @@ func (c *Comparator) sortStableIndices(vals []zed.Value) []uint32 { indices := make([]uint32, n) i64s := make([]int64, n) val0s := make([]zed.Value, n) - arena := zed.NewArena() - defer arena.Unref() - ectx := NewContext(arena) + ectx := NewContext() native := true for i := range indices { indices[i] = uint32(i) @@ -61,16 +59,12 @@ func (c *Comparator) sortStableIndices(vals []zed.Value) []uint32 { native = false } } - arena = zed.NewArena() - defer arena.Unref() - ectx = NewContext(arena) sort.SliceStable(indices, func(i, j int) bool { for k, expr := range c.exprs { iidx, jidx := indices[i], indices[j] if expr.Order == order.Desc { iidx, jidx = jidx, iidx } - arena.Reset() var ival, jval zed.Value if k == 0 { if native { @@ -85,7 +79,7 @@ func (c *Comparator) sortStableIndices(vals []zed.Value) []uint32 { ival = expr.Eval(ectx, vals[iidx]) jval = expr.Eval(ectx, vals[jidx]) } - if v := compareValues(arena, ival, jval, c.nullsMax); v != 0 { + if v := compareValues(ival, jval, c.nullsMax); v != 0 { return v < 0 } } @@ -128,7 +122,7 @@ type Comparator struct { // reverse reverses the sense of comparisons. func NewComparator(nullsMax bool, exprs ...SortEvaluator) *Comparator { return &Comparator{ - ectx: NewContext(zed.NewArena()), + ectx: NewContext(), exprs: slices.Clone(exprs), nullsMax: nullsMax, } @@ -157,20 +151,19 @@ func (m *missingAsNull) Eval(ectx Context, val zed.Value) zed.Value { // configuration. The result will be 0 if a==b, -1 if a < b, and +1 if a > b. func (c *Comparator) Compare(a, b zed.Value) int { for _, k := range c.exprs { - c.ectx.Arena().Reset() aval := k.Eval(c.ectx, a) bval := k.Eval(c.ectx, b) if k.Order == order.Desc { aval, bval = bval, aval } - if v := compareValues(c.ectx.Arena(), aval, bval, c.nullsMax); v != 0 { + if v := compareValues(aval, bval, c.nullsMax); v != 0 { return v } } return 0 } -func compareValues(arena *zed.Arena, a, b zed.Value, nullsMax bool) int { +func compareValues(a, b zed.Value, nullsMax bool) int { // Handle nulls according to nullsMax nullA := a.IsNull() nullB := b.IsNull() @@ -231,9 +224,9 @@ func compareValues(arena *zed.Arena, a, b zed.Value, nullsMax bool) int { if bit.Done() { return 1 } - aa := arena.New(innerType, ait.Next()) - bb := arena.New(innerType, bit.Next()) - if v := compareValues(arena, aa, bb, nullsMax); v != 0 { + aa := zed.NewValue(innerType, ait.Next()) + bb := zed.NewValue(innerType, bit.Next()) + if v := compareValues(aa, bb, nullsMax); v != 0 { return v } } diff --git a/runtime/sam/expr/sort_test.go b/runtime/sam/expr/sort_test.go index 88f98298cb..3ace453484 100644 --- a/runtime/sam/expr/sort_test.go +++ b/runtime/sam/expr/sort_test.go @@ -23,15 +23,12 @@ func BenchmarkSort(b *testing.B) { } for _, c := range cases { b.Run(zson.FormatType(c.typ), func(b *testing.B) { - arena := zed.NewArena() - defer arena.Unref() cmp := NewComparator(false, SortEvaluator{&This{}, order.Asc}) vals := make([]zed.Value, 1048576) for i := 0; i < b.N; i++ { b.StopTimer() - arena.Reset() for i := range vals { - vals[i] = arena.New(c.typ, c.bytes()) + vals[i] = zed.NewValue(c.typ, c.bytes()) } b.StartTimer() cmp.SortStable(vals) diff --git a/runtime/sam/expr/udf.go b/runtime/sam/expr/udf.go index 58ca097bdd..1e51e96f48 100644 --- a/runtime/sam/expr/udf.go +++ b/runtime/sam/expr/udf.go @@ -12,7 +12,7 @@ type UDF struct { Body Evaluator } -func (u *UDF) Call(ectx Context, args []zed.Value) zed.Value { +func (u *UDF) Call(ectx zed.Allocator, args []zed.Value) zed.Value { stack := 1 if f, ok := ectx.(*frame); ok { stack += f.stack @@ -22,13 +22,13 @@ func (u *UDF) Call(ectx Context, args []zed.Value) zed.Value { } // args must be cloned otherwise the values will be overwritten in // recursive calls. - f := &frame{ectx, stack, slices.Clone(args)} + f := &frame{stack: stack, vars: slices.Clone(args)} defer f.exit() return u.Body.Eval(f, zed.Null) } type frame struct { - Context + allocator stack int vars []zed.Value } diff --git a/runtime/sam/expr/values.go b/runtime/sam/expr/values.go index 827f2e339b..6e1870af4f 100644 --- a/runtime/sam/expr/values.go +++ b/runtime/sam/expr/values.go @@ -64,7 +64,7 @@ func (r *recordExpr) Eval(ectx Context, this zed.Value) zed.Value { // Return empty record instead of null record. bytes = []byte{} } - return ectx.Arena().New(r.typ, bytes) + return zed.NewValue(r.typ, bytes) } type RecordElem struct { @@ -113,7 +113,7 @@ func (r *recordSpreadExpr) Eval(ectx Context, this zed.Value) zed.Value { if !ok { fv = fieldValue{index: len(object)} } - fv.value = ectx.Arena().New(f.Type, it.Next()) + fv.value = zed.NewValue(f.Type, it.Next()) object[f.Name] = fv } } else { @@ -128,7 +128,7 @@ func (r *recordSpreadExpr) Eval(ectx Context, this zed.Value) zed.Value { } } if len(object) == 0 { - return ectx.Arena().New(r.zctx.MustLookupTypeRecord([]zed.Field{}), []byte{}) + return zed.NewValue(r.zctx.MustLookupTypeRecord([]zed.Field{}), []byte{}) } r.update(object) b := r.builder @@ -136,7 +136,7 @@ func (r *recordSpreadExpr) Eval(ectx Context, this zed.Value) zed.Value { for _, bytes := range r.bytes { b.Append(bytes) } - return ectx.Arena().New(r.cache, b.Bytes()) + return zed.NewValue(r.cache, b.Bytes()) } // update maps the object into the receiver's vals slice while also @@ -204,13 +204,13 @@ func (a *ArrayExpr) Eval(ectx Context, this zed.Value) zed.Value { a.collection.appendSpread(inner, val.Bytes()) } if len(a.collection.types) == 0 { - return ectx.Arena().New(a.zctx.LookupTypeArray(zed.TypeNull), []byte{}) + return zed.NewValue(a.zctx.LookupTypeArray(zed.TypeNull), []byte{}) } it := a.collection.iter(a.zctx) for !it.done() { it.appendNext(&a.builder) } - return ectx.Arena().New(a.zctx.LookupTypeArray(it.typ), a.builder.Bytes()) + return zed.NewValue(a.zctx.LookupTypeArray(it.typ), a.builder.Bytes()) } type SetExpr struct { @@ -244,13 +244,13 @@ func (a *SetExpr) Eval(ectx Context, this zed.Value) zed.Value { a.collection.appendSpread(inner, val.Bytes()) } if len(a.collection.types) == 0 { - return ectx.Arena().New(a.zctx.LookupTypeSet(zed.TypeNull), []byte{}) + return zed.NewValue(a.zctx.LookupTypeSet(zed.TypeNull), []byte{}) } it := a.collection.iter(a.zctx) for !it.done() { it.appendNext(&a.builder) } - return ectx.Arena().New(a.zctx.LookupTypeSet(it.typ), zed.NormalizeSet(a.builder.Bytes())) + return zed.NewValue(a.zctx.LookupTypeSet(it.typ), zed.NormalizeSet(a.builder.Bytes())) } type Entry struct { @@ -282,7 +282,7 @@ func (m *MapExpr) Eval(ectx Context, this zed.Value) zed.Value { } if len(m.keys.types) == 0 { typ := m.zctx.LookupTypeMap(zed.TypeNull, zed.TypeNull) - return ectx.Arena().New(typ, []byte{}) + return zed.NewValue(typ, []byte{}) } m.builder.Reset() kIter, vIter := m.keys.iter(m.zctx), m.vals.iter(m.zctx) @@ -292,7 +292,7 @@ func (m *MapExpr) Eval(ectx Context, this zed.Value) zed.Value { } bytes := m.builder.Bytes() typ := m.zctx.LookupTypeMap(kIter.typ, vIter.typ) - return ectx.Arena().New(typ, zed.NormalizeMap(bytes)) + return zed.NewValue(typ, zed.NormalizeMap(bytes)) } type collectionBuilder struct { diff --git a/runtime/sam/op/apply.go b/runtime/sam/op/apply.go index 62771fa1bb..472a89e024 100644 --- a/runtime/sam/op/apply.go +++ b/runtime/sam/op/apply.go @@ -30,12 +30,10 @@ func (a *applier) Pull(done bool) (zbuf.Batch, error) { a.resetter.Reset() return nil, err } - arena := zed.NewArena() - ectx := expr.NewContextWithVars(arena, batch.Vars()) vals := batch.Values() out := make([]zed.Value, 0, len(vals)) for i := range vals { - val := a.expr.Eval(ectx, vals[i]) + val := a.expr.Eval(batch, vals[i]) if val.IsError() { if val.IsQuiet() || val.IsMissing() { continue @@ -44,11 +42,8 @@ func (a *applier) Pull(done bool) (zbuf.Batch, error) { out = append(out, val) } if len(out) > 0 { - defer arena.Unref() - defer batch.Unref() - return zbuf.NewBatch(arena, out, batch, batch.Vars()), nil + return zbuf.NewBatch(batch, out), nil } - arena.Unref() batch.Unref() } } diff --git a/runtime/sam/op/explode/explode.go b/runtime/sam/op/explode/explode.go index 402cb01d81..9ca1485f30 100644 --- a/runtime/sam/op/explode/explode.go +++ b/runtime/sam/op/explode/explode.go @@ -31,30 +31,27 @@ func New(zctx *zed.Context, parent zbuf.Puller, args []expr.Evaluator, typ zed.T } func (o *Op) Pull(done bool) (zbuf.Batch, error) { - arena := zed.NewArena() - defer arena.Unref() for { batch, err := o.parent.Pull(done) if batch == nil || err != nil { o.resetter.Reset() return nil, err } - ectx := expr.NewContextWithVars(arena, batch.Vars()) vals := batch.Values() out := make([]zed.Value, 0, len(vals)) - for _, val := range vals { + for i := range vals { for _, arg := range o.args { - val := arg.Eval(ectx, val) + val := arg.Eval(batch, vals[i]) if val.IsError() { if !val.IsMissing() { - out = append(out, val) + out = append(out, val.Copy()) } continue } zed.Walk(val.Type(), val.Bytes(), func(typ zed.Type, body zcode.Bytes) error { if typ == o.typ && body != nil { bytes := zcode.Append(nil, body) - out = append(out, arena.New(o.outType, bytes)) + out = append(out, zed.NewValue(o.outType, bytes)) return zed.SkipContainer } return nil @@ -63,9 +60,8 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { } if len(out) > 0 { defer batch.Unref() - return zbuf.NewBatch(arena, out, batch, batch.Vars()), nil + return zbuf.NewBatch(batch, out), nil } - arena.Reset() batch.Unref() } } diff --git a/runtime/sam/op/exprswitch/exprswitch.go b/runtime/sam/op/exprswitch/exprswitch.go index b792b2e201..bfb8f9cf4c 100644 --- a/runtime/sam/op/exprswitch/exprswitch.go +++ b/runtime/sam/op/exprswitch/exprswitch.go @@ -46,12 +46,9 @@ func (s *ExprSwitch) AddCase(val *zed.Value) zbuf.Puller { } func (s *ExprSwitch) Forward(router *op.Router, batch zbuf.Batch) bool { - arena := zed.NewArena() - defer arena.Unref() - ectx := expr.NewContextWithVars(arena, batch.Vars()) vals := batch.Values() for i := range vals { - val := s.expr.Eval(ectx, vals[i]) + val := s.expr.Eval(batch, vals[i]) if val.IsMissing() { continue } @@ -69,7 +66,11 @@ func (s *ExprSwitch) Forward(router *op.Router, batch zbuf.Batch) bool { // ref the batch for each outgoing new batch. for _, c := range s.cases { if len(c.vals) > 0 { - out := zbuf.NewBatch(arena, c.vals, batch, batch.Vars()) + // XXX The new slice should come from the + // outgoing batch so we don't send these slices + // through GC. + batch.Ref() + out := zbuf.NewBatch(batch, c.vals) c.vals = nil if ok := router.Send(c.route, out, nil); !ok { return false @@ -77,7 +78,8 @@ func (s *ExprSwitch) Forward(router *op.Router, batch zbuf.Batch) bool { } } if c := s.defaultCase; c != nil && len(c.vals) > 0 { - out := zbuf.NewBatch(arena, c.vals, batch, batch.Vars()) + batch.Ref() + out := zbuf.NewArray(c.vals) c.vals = nil if ok := router.Send(c.route, out, nil); !ok { return false diff --git a/runtime/sam/op/fuse/fuser.go b/runtime/sam/op/fuse/fuser.go index 6ea5c86c4c..8c82cc41e1 100644 --- a/runtime/sam/op/fuse/fuser.go +++ b/runtime/sam/op/fuse/fuser.go @@ -22,7 +22,6 @@ type Fuser struct { uberSchema *agg.Schema shaper *expr.ConstShaper ectx expr.Context - arena *zed.Arena } // NewFuser returns a new Fuser. The Fuser buffers records in memory until @@ -34,8 +33,7 @@ func NewFuser(zctx *zed.Context, memMaxBytes int) *Fuser { memMaxBytes: memMaxBytes, types: make(map[zed.Type]struct{}), uberSchema: agg.NewSchema(zctx), - arena: zed.NewArena(), - ectx: expr.NewContext(zed.NewArena()), + ectx: expr.NewContext(), } } @@ -76,10 +74,9 @@ func (f *Fuser) stash(rec zed.Value) error { } } f.vals = nil - f.arena.Reset() return f.spiller.Write(rec) } - f.vals = append(f.vals, rec.Copy(f.arena)) + f.vals = append(f.vals, rec.Copy()) return nil } @@ -99,7 +96,6 @@ func (f *Fuser) Read() (*zed.Value, error) { if rec == nil || err != nil { return nil, err } - f.ectx.Arena().Reset() return f.shaper.Eval(f.ectx, *rec).Ptr(), nil } diff --git a/runtime/sam/op/groupby/groupby.go b/runtime/sam/op/groupby/groupby.go index d050f1a296..c33b0aa1eb 100644 --- a/runtime/sam/op/groupby/groupby.go +++ b/runtime/sam/op/groupby/groupby.go @@ -59,23 +59,18 @@ type Aggregator struct { valueCompare expr.CompareFn // to compare primary group keys for early key output keyCompare expr.CompareFn // compare the first key (used when input sorted) keysComparator *expr.Comparator // compare all keys - maxTableKey zed.Value - maxSpillKey zed.Value + maxTableKey *zed.Value + maxSpillKey *zed.Value inputDir order.Direction spiller *spill.MergeSort partialsIn bool partialsOut bool - - ectx *expr.Ctx - maxTableKeyArena *zed.Arena - maxSpillKeyArena *zed.Arena } type Row struct { - keyType int - groupval zed.Value // for sorting when input sorted - groupvalArena *zed.Arena - reducers valRow + keyType int + groupval zed.Value // for sorting when input sorted + reducers valRow } func NewAggregator(ctx context.Context, zctx *zed.Context, keyRefs, keyExprs, aggRefs []expr.Evaluator, aggs []*expr.Aggregator, builder *zed.RecordBuilder, limit int, inputDir order.Direction, partialsIn, partialsOut bool) (*Aggregator, error) { @@ -119,10 +114,6 @@ func NewAggregator(ctx context.Context, zctx *zed.Context, keyRefs, keyExprs, ag valueCompare: valueCompare, partialsIn: partialsIn, partialsOut: partialsOut, - - ectx: expr.NewContext(zed.NewArena()), - maxTableKeyArena: zed.NewArena(), - maxSpillKeyArena: zed.NewArena(), }, nil } @@ -336,14 +327,11 @@ func (a *Aggregator) Consume(batch zbuf.Batch, this zed.Value) error { // structure at output time, which is the new approach that will be // taken by the fix to #1701. - *a.ectx = *expr.NewContextWithVars(a.ectx.Arena(), batch.Vars()) - types := a.typeCache[:0] keyBytes := a.keyCache[:0] var prim zed.Value for i, keyExpr := range a.keyExprs { - a.ectx.Arena().Reset() - key := keyExpr.Eval(a.ectx, this) + key := keyExpr.Eval(batch, this) if key.IsQuiet() { return nil } @@ -370,20 +358,16 @@ func (a *Aggregator) Consume(batch zbuf.Batch, this zed.Value) error { } row = &Row{ keyType: keyType, + groupval: prim, reducers: newValRow(a.aggs), } - if !prim.IsZero() { - row.groupvalArena = zed.NewArena() - row.groupval = prim.Copy(row.groupvalArena) - } a.table[string(keyBytes)] = row } - a.ectx.Arena().Reset() if a.partialsIn { - row.reducers.consumeAsPartial(this, a.aggRefs, a.ectx) + row.reducers.consumeAsPartial(this, a.aggRefs, batch) } else { - row.reducers.apply(a.zctx, a.ectx, a.aggs, this) + row.reducers.apply(a.zctx, batch, a.aggs, this) } return nil } @@ -393,9 +377,8 @@ func (a *Aggregator) spillTable(eof bool, ref zbuf.Batch) error { if err != nil || batch == nil { return err } - defer batch.Unref() if a.spiller == nil { - a.spiller, err = spill.NewMergeSort(a.zctx, a.keysComparator) + a.spiller, err = spill.NewMergeSort(a.keysComparator) if err != nil { return err } @@ -406,11 +389,10 @@ func (a *Aggregator) spillTable(eof bool, ref zbuf.Batch) error { return err } if !eof && a.inputDir != 0 { - arena := zed.NewArena() - defer arena.Unref() - ectx := expr.NewContextWithVars(arena, ref.Vars()) - val := a.keyExprs[0].Eval(ectx, recs[len(recs)-1]) + val := a.keyExprs[0].Eval(batch, recs[len(recs)-1]) if !val.IsError() { + // pass volatile zed.Value since updateMaxSpillKey will make + // a copy if needed. a.updateMaxSpillKey(val) } } @@ -420,17 +402,15 @@ func (a *Aggregator) spillTable(eof bool, ref zbuf.Batch) error { // updateMaxTableKey is called with a volatile zed.Value to update the // max value seen in the table for the streaming logic when the input is sorted. func (a *Aggregator) updateMaxTableKey(val zed.Value) zed.Value { - if a.maxTableKey.IsZero() || a.valueCompare(val, a.maxTableKey) > 0 { - a.maxTableKeyArena.Reset() - a.maxTableKey = val.Copy(a.maxTableKeyArena) + if a.maxTableKey == nil || a.valueCompare(val, *a.maxTableKey) > 0 { + a.maxTableKey = val.Copy().Ptr() } - return a.maxTableKey + return *a.maxTableKey } func (a *Aggregator) updateMaxSpillKey(v zed.Value) { - if a.maxSpillKey.IsZero() || a.valueCompare(v, a.maxSpillKey) > 0 { - a.maxSpillKeyArena.Reset() - a.maxSpillKey = v.Copy(a.maxSpillKeyArena) + if a.maxSpillKey == nil || a.valueCompare(v, *a.maxSpillKey) > 0 { + a.maxSpillKey = v.Copy().Ptr() } } @@ -439,9 +419,6 @@ func (a *Aggregator) updateMaxSpillKey(v zed.Value) { // the input is sorted in the primary key, Results can be called // before eof, and keys that are completed will returned. func (a *Aggregator) nextResult(eof bool, batch zbuf.Batch) (zbuf.Batch, error) { - if batch == nil { - return nil, nil - } if a.spiller == nil { return a.readTable(eof, a.partialsOut, batch) } @@ -459,24 +436,21 @@ func (a *Aggregator) readSpills(eof bool, batch zbuf.Batch) (zbuf.Batch, error) if !eof && a.inputDir == 0 { return nil, nil } - arena := zed.NewArena() - defer arena.Unref() - ectx := expr.NewContextWithVars(arena, batch.Vars()) for len(recs) < op.BatchLen { if !eof && a.inputDir != 0 { - rec, err := a.spiller.Peek(arena) + rec, err := a.spiller.Peek() if err != nil { return nil, err } if rec == nil { break } - keyVal := a.keyExprs[0].Eval(ectx, *rec) - if !keyVal.IsError() && a.valueCompare(keyVal, a.maxSpillKey) >= 0 { + keyVal := a.keyExprs[0].Eval(batch, *rec) + if !keyVal.IsError() && a.valueCompare(keyVal, *a.maxSpillKey) >= 0 { break } } - rec, err := a.nextResultFromSpills(ectx) + rec, err := a.nextResultFromSpills(batch) if err != nil { return nil, err } @@ -488,7 +462,7 @@ func (a *Aggregator) readSpills(eof bool, batch zbuf.Batch) (zbuf.Batch, error) if len(recs) == 0 { return nil, nil } - return zbuf.NewBatch(arena, recs, batch, batch.Vars()), nil + return zbuf.NewBatch(batch, recs), nil } func (a *Aggregator) nextResultFromSpills(ectx expr.Context) (*zed.Value, error) { @@ -502,7 +476,7 @@ func (a *Aggregator) nextResultFromSpills(ectx expr.Context) (*zed.Value, error) row := newValRow(a.aggs) var firstRec *zed.Value for { - rec, err := a.spiller.Peek(ectx.Arena()) + rec, err := a.spiller.Peek() if err != nil { return nil, err } @@ -510,7 +484,7 @@ func (a *Aggregator) nextResultFromSpills(ectx expr.Context) (*zed.Value, error) break } if firstRec == nil { - firstRec = rec.Copy(ectx.Arena()).Ptr() + firstRec = rec.Copy().Ptr() } else if a.keysComparator.Compare(*firstRec, *rec) != 0 { break } @@ -533,9 +507,9 @@ func (a *Aggregator) nextResultFromSpills(ectx expr.Context) (*zed.Value, error) for _, f := range row { var v zed.Value if a.partialsOut { - v = f.ResultAsPartial(a.zctx, ectx.Arena()) + v = f.ResultAsPartial(a.zctx) } else { - v = f.Result(a.zctx, ectx.Arena()) + v = f.Result(a.zctx) } types = append(types, v.Type()) a.builder.Append(v.Bytes()) @@ -545,7 +519,7 @@ func (a *Aggregator) nextResultFromSpills(ectx expr.Context) (*zed.Value, error) if err != nil { return nil, err } - return ectx.Arena().New(typ, bytes).Ptr(), nil + return zed.NewValue(typ, bytes).Ptr(), nil } // readTable returns a slice of records from the in-memory groupby @@ -554,14 +528,12 @@ func (a *Aggregator) nextResultFromSpills(ectx expr.Context) (*zed.Value, error) // If partialsOut is true, it returns partial aggregation results as // defined by each agg.Function.ResultAsPartial() method. func (a *Aggregator) readTable(flush, partialsOut bool, batch zbuf.Batch) (zbuf.Batch, error) { - arena := zed.NewArena() - defer arena.Unref() var recs []zed.Value for key, row := range a.table { if !flush && a.valueCompare == nil { panic("internal bug: tried to fetch completed tuples on non-sorted input") } - if !flush && a.valueCompare(row.groupval, a.maxTableKey) >= 0 { + if !flush && a.valueCompare(row.groupval, *a.maxTableKey) >= 0 { continue } // To build the output record, we spin over the key values @@ -584,9 +556,9 @@ func (a *Aggregator) readTable(flush, partialsOut bool, batch zbuf.Batch) (zbuf. for _, f := range row.reducers { var v zed.Value if partialsOut { - v = f.ResultAsPartial(a.zctx, arena) + v = f.ResultAsPartial(a.zctx) } else { - v = f.Result(a.zctx, arena) + v = f.Result(a.zctx) } types = append(types, v.Type()) a.builder.Append(v.Bytes()) @@ -596,7 +568,7 @@ func (a *Aggregator) readTable(flush, partialsOut bool, batch zbuf.Batch) (zbuf. if err != nil { return nil, err } - recs = append(recs, arena.New(typ, zv)) + recs = append(recs, zed.NewValue(typ, zv)) // Delete entries from the table as we create records, so // the freed enries can be GC'd incrementally as we shift // state from the table to the records. Otherwise, when @@ -608,7 +580,7 @@ func (a *Aggregator) readTable(flush, partialsOut bool, batch zbuf.Batch) (zbuf. if len(recs) == 0 { return nil, nil } - return zbuf.NewBatch(arena, recs, batch, batch.Vars()), nil + return zbuf.NewBatch(batch, recs), nil } func (a *Aggregator) lookupRecordType(types []zed.Type) *zed.TypeRecord { diff --git a/runtime/sam/op/groupby/groupby_test.go b/runtime/sam/op/groupby/groupby_test.go index faf8015e37..93724525f4 100644 --- a/runtime/sam/op/groupby/groupby_test.go +++ b/runtime/sam/op/groupby/groupby_test.go @@ -102,9 +102,6 @@ func TestGroupbyStreamingSpill(t *testing.T) { data = append(data, fmt.Sprintf("{ts:%s,ip:1.1.1.%d}", nano.Unix(int64(t), 0), i%uniqueIpsPerTs)) } - arena := zed.NewArena() - defer arena.Unref() - runOne := func(inputSortKey string) []string { proc, _, err := compiler.Parse("count() by every(1s), ip") assert.NoError(t, err) diff --git a/runtime/sam/op/head/head.go b/runtime/sam/op/head/head.go index 5c0b21e873..ee775ad5d1 100644 --- a/runtime/sam/op/head/head.go +++ b/runtime/sam/op/head/head.go @@ -61,6 +61,5 @@ again: return nil, err } o.count = o.limit - defer batch.Unref() - return zbuf.WrapBatch(batch, vals[:remaining]), nil + return zbuf.NewBatch(batch, vals[:remaining]), nil } diff --git a/runtime/sam/op/join/join.go b/runtime/sam/op/join/join.go index 2985451b3d..d8047c2dae 100644 --- a/runtime/sam/op/join/join.go +++ b/runtime/sam/op/join/join.go @@ -31,9 +31,6 @@ type Op struct { joinKey *zed.Value joinSet []zed.Value types map[int]map[int]*zed.TypeRecord - - joinKeyArena *zed.Arena - joinSetArena *zed.Arena } func New(rctx *runtime.Context, anti, inner bool, left, right zbuf.Puller, leftKey, rightKey expr.Evaluator, @@ -63,21 +60,19 @@ func New(rctx *runtime.Context, anti, inner bool, left, right zbuf.Puller, leftK } ctx, cancel := context.WithCancel(rctx.Context) return &Op{ - rctx: rctx, - anti: anti, - inner: inner, - ctx: ctx, - cancel: cancel, - getLeftKey: leftKey, - getRightKey: rightKey, - left: newPuller(left, ctx), - right: zio.NewPeeker(newPuller(right, ctx)), - resetter: resetter, - compare: expr.NewValueCompareFn(o, true), - cutter: expr.NewCutter(rctx.Zctx, lhs, rhs), - types: make(map[int]map[int]*zed.TypeRecord), - joinKeyArena: zed.NewArena(), - joinSetArena: zed.NewArena(), + rctx: rctx, + anti: anti, + inner: inner, + ctx: ctx, + cancel: cancel, + getLeftKey: leftKey, + getRightKey: rightKey, + left: newPuller(left, ctx), + right: zio.NewPeeker(newPuller(right, ctx)), + resetter: resetter, + compare: expr.NewValueCompareFn(o, true), + cutter: expr.NewCutter(rctx.Zctx, lhs, rhs), + types: make(map[int]map[int]*zed.TypeRecord), }, nil } @@ -88,10 +83,9 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { go o.left.run() go o.right.Reader.(*puller).run() }) - arena := zed.NewArena() - // See #3366 - ectx := expr.NewContext(arena) var out []zed.Value + // See #3366 + ectx := expr.NewContext() for { leftRec, err := o.left.Read() if err != nil { @@ -103,7 +97,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { return nil, nil } //XXX See issue #3427. - return zbuf.NewArray(arena, out), nil + return zbuf.NewArray(out), nil } key := o.getLeftKey.Eval(ectx, *leftRec) if key.IsMissing() { @@ -112,7 +106,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { // left records that can eval the key expression. continue } - rightRecs, err := o.getJoinSet(ectx, key) + rightRecs, err := o.getJoinSet(key) if err != nil { return nil, err } @@ -120,7 +114,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { // Nothing to add to the left join. // Accumulate this record for an outer join. if !o.inner { - out = append(out, leftRec.Copy(arena)) + out = append(out, leftRec.Copy()) } continue } @@ -138,7 +132,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { // release the batch with and bypass GC. for _, rightRec := range rightRecs { cutRec := o.cutter.Eval(ectx, rightRec) - rec, err := o.splice(arena, *leftRec, cutRec) + rec, err := o.splice(*leftRec, cutRec) if err != nil { return nil, err } @@ -147,11 +141,12 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { } } -func (o *Op) getJoinSet(ectx expr.Context, leftKey zed.Value) ([]zed.Value, error) { +func (o *Op) getJoinSet(leftKey zed.Value) ([]zed.Value, error) { if o.joinKey != nil && o.compare(leftKey, *o.joinKey) == 0 { return o.joinSet, nil } // See #3366 + ectx := expr.NewContext() for { rec, err := o.right.Peek() if err != nil || rec == nil { @@ -164,9 +159,13 @@ func (o *Op) getJoinSet(ectx expr.Context, leftKey zed.Value) ([]zed.Value, erro } cmp := o.compare(leftKey, rightKey) if cmp == 0 { - o.joinKeyArena.Reset() - o.joinKey = leftKey.Copy(o.joinKeyArena).Ptr() - o.joinSet, err = o.readJoinSet(ectx, o.joinKey) + // Copy leftKey.Bytes since it might get reused. + if o.joinKey == nil { + o.joinKey = leftKey.Copy().Ptr() + } else { + o.joinKey.CopyFrom(leftKey) + } + o.joinSet, err = o.readJoinSet(o.joinKey) return o.joinSet, err } if cmp < 0 { @@ -185,10 +184,10 @@ func (o *Op) getJoinSet(ectx expr.Context, leftKey zed.Value) ([]zed.Value, erro // fillJoinSet is called when a join key has been found that matches // the current lefthand key. It returns the all the subsequent records // from the righthand stream that match this key. -func (o *Op) readJoinSet(ectx expr.Context, joinKey *zed.Value) ([]zed.Value, error) { - o.joinSetArena.Reset() - // See #3366 +func (o *Op) readJoinSet(joinKey *zed.Value) ([]zed.Value, error) { var recs []zed.Value + // See #3366 + ectx := expr.NewContext() for { rec, err := o.right.Peek() if err != nil { @@ -205,7 +204,7 @@ func (o *Op) readJoinSet(ectx expr.Context, joinKey *zed.Value) ([]zed.Value, er if o.compare(key, *joinKey) != 0 { return recs, nil } - recs = append(recs, rec.Copy(o.joinSetArena)) + recs = append(recs, rec.Copy()) o.right.Read() } } @@ -252,9 +251,9 @@ func (o *Op) combinedType(left, right *zed.TypeRecord) (*zed.TypeRecord, error) return typ, nil } -func (o *Op) splice(arena *zed.Arena, left, right zed.Value) (zed.Value, error) { - left = left.Under(arena) - right = right.Under(arena) +func (o *Op) splice(left, right zed.Value) (zed.Value, error) { + left = left.Under() + right = right.Under() typ, err := o.combinedType(zed.TypeRecordOf(left.Type()), zed.TypeRecordOf(right.Type())) if err != nil { return zed.Null, err @@ -263,5 +262,5 @@ func (o *Op) splice(arena *zed.Arena, left, right zed.Value) (zed.Value, error) bytes := make([]byte, n+len(right.Bytes())) copy(bytes, left.Bytes()) copy(bytes[n:], right.Bytes()) - return arena.New(typ, bytes), nil + return zed.NewValue(typ, bytes), nil } diff --git a/runtime/sam/op/load/load.go b/runtime/sam/op/load/load.go index 136420e9d0..afa64dbbf2 100644 --- a/runtime/sam/op/load/load.go +++ b/runtime/sam/op/load/load.go @@ -66,8 +66,6 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { if err != nil { return nil, err } - arena := zed.NewArena() - defer arena.Unref() - val := arena.NewBytes(commitID[:]) - return zbuf.NewArray(arena, []zed.Value{val}), nil + val := zed.NewBytes(commitID[:]) + return zbuf.NewArray([]zed.Value{val}), nil } diff --git a/runtime/sam/op/merge/merge.go b/runtime/sam/op/merge/merge.go index dfa81b55b4..f54515a319 100644 --- a/runtime/sam/op/merge/merge.go +++ b/runtime/sam/op/merge/merge.go @@ -75,8 +75,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { // way, it's safe to return min's remaining values as a batch. batch := min.batch if len(min.vals) < len(batch.Values()) { - defer batch.Unref() - batch = zbuf.WrapBatch(batch, min.vals) + batch = zbuf.NewArray(min.vals) } ok, err := min.replenish() if err != nil { diff --git a/runtime/sam/op/meta/deleter.go b/runtime/sam/op/meta/deleter.go index 43178967af..abf6329456 100644 --- a/runtime/sam/op/meta/deleter.go +++ b/runtime/sam/op/meta/deleter.go @@ -9,31 +9,34 @@ import ( "github.com/brimdata/zed/runtime" "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zbuf" + "github.com/brimdata/zed/zson" "github.com/segmentio/ksuid" ) type Deleter struct { - parent zbuf.Puller - scanner zbuf.Puller - filter zbuf.Filter - pruner expr.Evaluator - rctx *runtime.Context - pool *lake.Pool - progress *zbuf.Progress - done bool - err error - deletes *sync.Map + parent zbuf.Puller + scanner zbuf.Puller + filter zbuf.Filter + pruner expr.Evaluator + rctx *runtime.Context + pool *lake.Pool + progress *zbuf.Progress + unmarshaler *zson.UnmarshalZNGContext + done bool + err error + deletes *sync.Map } func NewDeleter(rctx *runtime.Context, parent zbuf.Puller, pool *lake.Pool, filter zbuf.Filter, pruner expr.Evaluator, progress *zbuf.Progress, deletes *sync.Map) *Deleter { return &Deleter{ - parent: parent, - filter: filter, - pruner: pruner, - rctx: rctx, - pool: pool, - progress: progress, - deletes: deletes, + parent: parent, + filter: filter, + pruner: pruner, + rctx: rctx, + pool: pool, + progress: progress, + unmarshaler: zson.NewZNGUnmarshaler(), + deletes: deletes, } } @@ -87,25 +90,21 @@ func (d *Deleter) nextDeletion() (zbuf.Puller, error) { if hasDeletes, err := d.hasDeletes(vals[0]); err != nil { return nil, err } else if !hasDeletes { - batch.Unref() continue } // Use a no-op progress so stats are not inflated. var progress zbuf.Progress - scanner, object, err := newScanner(d.rctx.Context, d.rctx.Zctx, d.pool, d.pruner, d.filter, &progress, vals[0]) + scanner, object, err := newScanner(d.rctx.Context, d.rctx.Zctx, d.pool, d.unmarshaler, d.pruner, d.filter, &progress, vals[0]) if err != nil { return nil, err } - batch.Unref() d.deleteObject(object.ID) return scanner, nil } } func (d *Deleter) hasDeletes(val zed.Value) (bool, error) { - arena := zed.NewArena() - defer arena.Unref() - scanner, object, err := newScanner(d.rctx.Context, d.rctx.Zctx, d.pool, d.pruner, d.filter, d.progress, val) + scanner, object, err := newScanner(d.rctx.Context, d.rctx.Zctx, d.pool, d.unmarshaler, d.pruner, d.filter, d.progress, val) if err != nil { return false, err } diff --git a/runtime/sam/op/meta/lister.go b/runtime/sam/op/meta/lister.go index 33928ef612..a5ee7d1a44 100644 --- a/runtime/sam/op/meta/lister.go +++ b/runtime/sam/op/meta/lister.go @@ -81,18 +81,16 @@ func (l *Lister) Pull(done bool) (zbuf.Batch, error) { if l.objects == nil { l.objects = initObjectScan(l.snap, l.pool.SortKeys.Primary()) } - arena := zed.NewArena() for len(l.objects) != 0 { - arena.Reset() o := l.objects[0] l.objects = l.objects[1:] - val, err := l.marshaler.Marshal(arena, o) + val, err := l.marshaler.Marshal(o) if err != nil { l.err = err return nil, err } if !l.pruner.prune(val) { - return zbuf.NewArray(arena, []zed.Value{val}), nil + return zbuf.NewArray([]zed.Value{val}), nil } } return nil, nil diff --git a/runtime/sam/op/meta/pruner.go b/runtime/sam/op/meta/pruner.go index 80309cb7dd..0cb22ebfa7 100644 --- a/runtime/sam/op/meta/pruner.go +++ b/runtime/sam/op/meta/pruner.go @@ -13,7 +13,7 @@ type pruner struct { func newPruner(e expr.Evaluator) *pruner { return &pruner{ pred: e, - ectx: expr.NewContext(zed.NewArena()), + ectx: expr.NewContext(), } } @@ -21,7 +21,6 @@ func (p *pruner) prune(val zed.Value) bool { if p == nil { return false } - p.ectx.Arena().Reset() result := p.pred.Eval(p.ectx, val) return result.Type() == zed.TypeBool && result.Bool() } diff --git a/runtime/sam/op/meta/scanner.go b/runtime/sam/op/meta/scanner.go index 26fd85517a..3ea86e14d8 100644 --- a/runtime/sam/op/meta/scanner.go +++ b/runtime/sam/op/meta/scanner.go @@ -16,21 +16,20 @@ import ( ) func NewLakeMetaScanner(ctx context.Context, zctx *zed.Context, r *lake.Root, meta string) (zbuf.Scanner, error) { - arena := zed.NewArena() var vals []zed.Value var err error switch meta { case "pools": - vals, err = r.BatchifyPools(ctx, zctx, arena, nil) + vals, err = r.BatchifyPools(ctx, zctx, nil) case "branches": - vals, err = r.BatchifyBranches(ctx, zctx, arena, nil) + vals, err = r.BatchifyBranches(ctx, zctx, nil) default: return nil, fmt.Errorf("unknown lake metadata type: %q", meta) } if err != nil { return nil, err } - return zbuf.NewScanner(ctx, zbuf.NewArray(arena, vals), nil) + return zbuf.NewScanner(ctx, zbuf.NewArray(vals), nil) } func NewPoolMetaScanner(ctx context.Context, zctx *zed.Context, r *lake.Root, poolID ksuid.KSUID, meta string) (zbuf.Scanner, error) { @@ -38,20 +37,19 @@ func NewPoolMetaScanner(ctx context.Context, zctx *zed.Context, r *lake.Root, po if err != nil { return nil, err } - arena := zed.NewArena() var vals []zed.Value switch meta { case "branches": m := zson.NewZNGMarshalerWithContext(zctx) m.Decorate(zson.StylePackage) - vals, err = p.BatchifyBranches(ctx, zctx, arena, nil, m, nil) + vals, err = p.BatchifyBranches(ctx, zctx, nil, m, nil) if err != nil { return nil, err } default: return nil, fmt.Errorf("unknown pool metadata type: %q", meta) } - return zbuf.NewScanner(ctx, zbuf.NewArray(arena, vals), nil) + return zbuf.NewScanner(ctx, zbuf.NewArray(vals), nil) } func NewCommitMetaScanner(ctx context.Context, zctx *zed.Context, r *lake.Root, poolID, commit ksuid.KSUID, meta string, pruner expr.Evaluator) (zbuf.Puller, error) { @@ -77,12 +75,11 @@ func NewCommitMetaScanner(ctx context.Context, zctx *zed.Context, r *lake.Root, } return zbuf.NewScanner(ctx, zbuf.PullerReader(slicer), nil) case "log": - arena := zed.NewArena() - tips, err := p.BatchifyBranchTips(ctx, zctx, arena, nil) + tips, err := p.BatchifyBranchTips(ctx, zctx, nil) if err != nil { return nil, err } - tipsScanner, err := zbuf.NewScanner(ctx, zbuf.NewArray(arena, tips), nil) + tipsScanner, err := zbuf.NewScanner(ctx, zbuf.NewArray(tips), nil) if err != nil { return nil, err } @@ -115,7 +112,6 @@ func NewCommitMetaScanner(ctx context.Context, zctx *zed.Context, r *lake.Root, } func objectReader(ctx context.Context, zctx *zed.Context, snap commits.View, order order.Which) (zio.Reader, error) { - arena := zed.NewArena() objects := snap.Select(nil, order) m := zson.NewZNGMarshalerWithContext(zctx) m.Decorate(zson.StylePackage) @@ -123,8 +119,7 @@ func objectReader(ctx context.Context, zctx *zed.Context, snap commits.View, ord if len(objects) == 0 { return nil, nil } - arena.Reset() - val, err := m.Marshal(arena, objects[0]) + val, err := m.Marshal(objects[0]) objects = objects[1:] return &val, err }), nil diff --git a/runtime/sam/op/meta/sequence.go b/runtime/sam/op/meta/sequence.go index 3fb4c8f064..d788c7d50f 100644 --- a/runtime/sam/op/meta/sequence.go +++ b/runtime/sam/op/meta/sequence.go @@ -21,25 +21,27 @@ import ( // SequenceScanner implements an op that pulls metadata partitions to scan // from its parent and for each partition, scans the object. type SequenceScanner struct { - parent zbuf.Puller - scanner zbuf.Puller - filter zbuf.Filter - pruner expr.Evaluator - rctx *runtime.Context - pool *lake.Pool - progress *zbuf.Progress - done bool - err error + parent zbuf.Puller + scanner zbuf.Puller + filter zbuf.Filter + pruner expr.Evaluator + rctx *runtime.Context + pool *lake.Pool + progress *zbuf.Progress + unmarshaler *zson.UnmarshalZNGContext + done bool + err error } func NewSequenceScanner(rctx *runtime.Context, parent zbuf.Puller, pool *lake.Pool, filter zbuf.Filter, pruner expr.Evaluator, progress *zbuf.Progress) *SequenceScanner { return &SequenceScanner{ - rctx: rctx, - parent: parent, - filter: filter, - pruner: pruner, - pool: pool, - progress: progress, + rctx: rctx, + parent: parent, + filter: filter, + pruner: pruner, + pool: pool, + progress: progress, + unmarshaler: zson.NewZNGUnmarshaler(), } } @@ -69,12 +71,11 @@ func (s *SequenceScanner) Pull(done bool) (zbuf.Batch, error) { s.close(err) return nil, err } - s.scanner, _, err = newScanner(s.rctx.Context, s.rctx.Zctx, s.pool, s.pruner, s.filter, s.progress, vals[0]) + s.scanner, _, err = newScanner(s.rctx.Context, s.rctx.Zctx, s.pool, s.unmarshaler, s.pruner, s.filter, s.progress, vals[0]) if err != nil { s.close(err) return nil, err } - batch.Unref() } batch, err := s.scanner.Pull(false) if err != nil { @@ -154,14 +155,11 @@ func (s *SearchScanner) Pull(done bool) (zbuf.Batch, error) { } } -func newScanner(ctx context.Context, zctx *zed.Context, pool *lake.Pool, pruner expr.Evaluator, filter zbuf.Filter, progress *zbuf.Progress, val zed.Value) (zbuf.Puller, *data.Object, error) { +func newScanner(ctx context.Context, zctx *zed.Context, pool *lake.Pool, u *zson.UnmarshalZNGContext, pruner expr.Evaluator, filter zbuf.Filter, progress *zbuf.Progress, val zed.Value) (zbuf.Puller, *data.Object, error) { named, ok := val.Type().(*zed.TypeNamed) if !ok { return nil, nil, errors.New("system error: SequenceScanner encountered unnamed object") } - arena := zed.NewArena() - u := zson.NewZNGUnmarshaler() - u.SetContext(zctx, arena) var objects []*data.Object if named.Name == "data.Object" { var object data.Object @@ -176,9 +174,6 @@ func newScanner(ctx context.Context, zctx *zed.Context, pool *lake.Pool, pruner } objects = part.Objects } - for _, o := range objects { - o.Arena = arena - } scanner, err := newObjectsScanner(ctx, zctx, pool, objects, pruner, filter, progress) return scanner, objects[0], err } diff --git a/runtime/sam/op/meta/slicer.go b/runtime/sam/op/meta/slicer.go index 78339a2d91..4643552a9b 100644 --- a/runtime/sam/op/meta/slicer.go +++ b/runtime/sam/op/meta/slicer.go @@ -18,11 +18,9 @@ import ( // them into overlapping object Slices forming a sequence of // non-overlapping Partitions. type Slicer struct { - zctx *zed.Context parent zbuf.Puller marshaler *zson.MarshalZNGContext unmarshaler *zson.UnmarshalZNGContext - arena *zed.Arena objects []*data.Object cmp expr.CompareFn min *zed.Value @@ -34,7 +32,6 @@ func NewSlicer(parent zbuf.Puller, zctx *zed.Context) *Slicer { m := zson.NewZNGMarshalerWithContext(zctx) m.Decorate(zson.StylePackage) return &Slicer{ - zctx: zctx, parent: parent, marshaler: m, unmarshaler: zson.NewZNGUnmarshaler(), @@ -69,12 +66,9 @@ func (s *Slicer) Pull(done bool) (zbuf.Batch, error) { return nil, errors.New("system error: Slicer encountered multi-valued batch") } var object data.Object - object.Arena = zed.NewArena() - s.unmarshaler.SetContext(s.zctx, object.Arena) if err := s.unmarshaler.Unmarshal(vals[0], &object); err != nil { return nil, err } - batch.Unref() if batch, err := s.stash(&object); batch != nil || err != nil { return batch, err } @@ -98,8 +92,7 @@ func (s *Slicer) nextPartition() (zbuf.Batch, error) { max = o.Max } } - arena := zed.NewArena() - val, err := s.marshaler.Marshal(arena, &Partition{ + val, err := s.marshaler.Marshal(&Partition{ Min: min, Max: max, Objects: s.objects, @@ -108,7 +101,7 @@ func (s *Slicer) nextPartition() (zbuf.Batch, error) { if err != nil { return nil, err } - return zbuf.NewArray(arena, []zed.Value{val}), nil + return zbuf.NewArray([]zed.Value{val}), nil } func (s *Slicer) stash(o *data.Object) (zbuf.Batch, error) { @@ -129,10 +122,10 @@ func (s *Slicer) stash(o *data.Object) (zbuf.Batch, error) { } s.objects = append(s.objects, o) if s.min == nil || s.cmp(*s.min, o.Min) > 0 { - s.min = &o.Min + s.min = o.Min.Copy().Ptr() } if s.max == nil || s.cmp(*s.max, o.Max) < 0 { - s.max = &o.Max + s.max = o.Max.Copy().Ptr() } return batch, nil } diff --git a/runtime/sam/op/router.go b/runtime/sam/op/router.go index 4b359ad661..bfdef13147 100644 --- a/runtime/sam/op/router.go +++ b/runtime/sam/op/router.go @@ -1,10 +1,10 @@ package op import ( + "context" "slices" "sync" - "github.com/brimdata/zed/runtime" "github.com/brimdata/zed/runtime/sam/expr" "github.com/brimdata/zed/zbuf" ) @@ -15,16 +15,16 @@ type Selector interface { } type Router struct { - rctx *runtime.Context + ctx context.Context parent zbuf.Puller selector Selector routes []*route once sync.Once } -func NewRouter(rctx *runtime.Context, parent zbuf.Puller) *Router { +func NewRouter(ctx context.Context, parent zbuf.Puller) *Router { return &Router{ - rctx: rctx, + ctx: ctx, parent: NewCatcher(parent), } } @@ -116,7 +116,7 @@ func (r *Router) sendEOS(err error) bool { // This path was about to be blocked with a done so // just mark it blocked now. p.blocked = true - case <-r.rctx.Done(): + case <-r.ctx.Done(): return false } } @@ -151,7 +151,7 @@ func (r *Router) Send(p zbuf.Puller, b zbuf.Batch, err error) bool { b.Unref() to.blocked = true return true - case <-r.rctx.Done(): + case <-r.ctx.Done(): return false } } @@ -173,14 +173,14 @@ func (r *route) Pull(done bool) (zbuf.Batch, error) { select { case r.doneCh <- struct{}{}: return nil, nil - case <-r.router.rctx.Done(): - return nil, r.router.rctx.Err() + case <-r.router.ctx.Done(): + return nil, r.router.ctx.Err() } } select { case result := <-r.resultCh: return result.Batch, result.Err - case <-r.router.rctx.Done(): - return nil, r.router.rctx.Err() + case <-r.router.ctx.Done(): + return nil, r.router.ctx.Err() } } diff --git a/runtime/sam/op/shape/shaper.go b/runtime/sam/op/shape/shaper.go index 78e8276f2e..06a115b512 100644 --- a/runtime/sam/op/shape/shaper.go +++ b/runtime/sam/op/shape/shaper.go @@ -21,8 +21,6 @@ type Shaper struct { hash maphash.Hash val zed.Value vals []zed.Value - readArena *zed.Arena - valsArena *zed.Arena } type anchor struct { @@ -86,11 +84,9 @@ func (i *integer) check(val zed.Value) { } func (a *anchor) updateInts(rec *zed.Value) error { - arena := zed.NewArena() - defer arena.Unref() it := rec.Bytes().Iter() for k, f := range rec.Fields() { - a.integers[k].check(arena.New(f.Type, it.Next())) + a.integers[k].check(zed.NewValue(f.Type, it.Next())) } return nil } @@ -124,8 +120,6 @@ func NewShaper(zctx *zed.Context, memMaxBytes int) *Shaper { anchors: make(map[uint64]*anchor), typeAnchor: make(map[zed.Type]*anchor), recode: make(map[zed.Type]*zed.TypeRecord), - readArena: zed.NewArena(), - valsArena: zed.NewArena(), } } @@ -250,7 +244,7 @@ func (s *Shaper) stash(rec zed.Value) error { s.vals = nil return s.spiller.Write(rec) } - s.vals = append(s.vals, rec.Copy(s.valsArena)) + s.vals = append(s.vals, rec.Copy()) return nil } @@ -274,8 +268,7 @@ func (s *Shaper) Read() (*zed.Value, error) { } typ = targetType } - s.readArena.Reset() - s.val = s.readArena.New(typ, bytes) + s.val = zed.NewValue(typ, bytes) return &s.val, nil } diff --git a/runtime/sam/op/sort/sort.go b/runtime/sam/op/sort/sort.go index f58c7dd466..051d216502 100644 --- a/runtime/sam/op/sort/sort.go +++ b/runtime/sam/op/sort/sort.go @@ -1,8 +1,6 @@ package sort import ( - "fmt" - "runtime/debug" "sync" "github.com/brimdata/zed" @@ -27,6 +25,7 @@ type Op struct { reverse bool fieldResolvers []expr.SortEvaluator + lastBatch zbuf.Batch once sync.Once resultCh chan op.Result comparator *expr.Comparator @@ -63,6 +62,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { } func (o *Op) run() { + defer close(o.resultCh) var spiller *spill.MergeSort defer func() { if spiller != nil { @@ -70,13 +70,8 @@ func (o *Op) run() { } // Tell o.rctx.Cancel that we've finished our cleanup. o.rctx.WaitGroup.Done() - if r := recover(); r != nil { - o.sendResult(nil, fmt.Errorf("panic: %+v\n%s", r, debug.Stack())) - } - close(o.resultCh) }() var nbytes int - var batches []zbuf.Batch var out []zed.Value for { batch, err := o.parent.Pull(false) @@ -89,7 +84,7 @@ func (o *Op) run() { if batch == nil { if spiller == nil { if len(out) > 0 { - if ok := o.send(batches, out); !ok { + if ok := o.send(out); !ok { return } } @@ -97,7 +92,6 @@ func (o *Op) run() { return } nbytes = 0 - batches = nil out = nil continue } @@ -108,7 +102,6 @@ func (o *Op) run() { } spiller = nil nbytes = 0 - batches = nil out = nil continue } @@ -119,12 +112,11 @@ func (o *Op) run() { spiller.Cleanup() spiller = nil nbytes = 0 - unrefBatches(batches) - batches = nil out = nil continue } - batches = append(batches, batch) + // Safe because batch.Unref is never called. + o.lastBatch = batch var delta int out, delta = o.append(out, batch) if o.comparator == nil && len(out) > 0 { @@ -135,12 +127,11 @@ func (o *Op) run() { continue } if spiller == nil { - spiller, err = spill.NewMergeSort(o.rctx.Zctx, o.comparator) + spiller, err = spill.NewMergeSort(o.comparator) if err != nil { if ok := o.sendResult(nil, err); !ok { return } - batches = nil out = nil nbytes = 0 continue @@ -151,21 +142,15 @@ func (o *Op) run() { return } } - // While it's good to unref batches if this isn't here the go runtime - // will start garbage collecting batches before Spill is complete - // because batches getting set to nil. - unrefBatches(batches) - batches = nil out = nil nbytes = 0 } } // send sorts vals in memory and sends the result downstream. -func (o *Op) send(batches []zbuf.Batch, vals []zed.Value) bool { +func (o *Op) send(vals []zed.Value) bool { o.comparator.SortStable(vals) - out := zbuf.WrapBatch(batches[0], vals) - out.(interface{ AddBatches(batches ...zbuf.Batch) }).AddBatches(batches[1:]...) + out := zbuf.NewBatch(o.lastBatch, vals) return o.sendResult(out, nil) } diff --git a/runtime/sam/op/spill/merge.go b/runtime/sam/op/spill/merge.go index e17cdd7cb5..8e8e756f62 100644 --- a/runtime/sam/op/spill/merge.go +++ b/runtime/sam/op/spill/merge.go @@ -37,7 +37,7 @@ func TempFile() (*os.File, error) { // NewMergeSort returns a MergeSort to implement external merge sorts of a large // zng record stream. It creates a temporary directory to hold the collection // of spilled chunks. Call Cleanup to remove it. -func NewMergeSort(zctx *zed.Context, comparator *expr.Comparator) (*MergeSort, error) { +func NewMergeSort(comparator *expr.Comparator) (*MergeSort, error) { tempDir, err := TempDir() if err != nil { return nil, err @@ -45,7 +45,7 @@ func NewMergeSort(zctx *zed.Context, comparator *expr.Comparator) (*MergeSort, e return &MergeSort{ comparator: comparator, tempDir: tempDir, - zctx: zctx, + zctx: zed.NewContext(), }, nil } @@ -99,11 +99,11 @@ func goWithContext(ctx context.Context, f func()) error { // Peek returns the next record without advancing the reader. The record stops // being valid at the next read call. -func (r *MergeSort) Peek(arena *zed.Arena) (*zed.Value, error) { +func (r *MergeSort) Peek() (*zed.Value, error) { if r.Len() == 0 { return nil, nil } - return r.runs[0].nextRecord.Copy(arena).Ptr(), nil + return r.runs[0].nextRecord, nil } // Read returns the smallest record (per the comparison function provided to MewMergeSort) diff --git a/runtime/sam/op/spill/peeker.go b/runtime/sam/op/spill/peeker.go index 367b761698..1fdc3e64d1 100644 --- a/runtime/sam/op/spill/peeker.go +++ b/runtime/sam/op/spill/peeker.go @@ -11,7 +11,6 @@ type peeker struct { *File nextRecord *zed.Value ordinal int - arena *zed.Arena } func newPeeker(ctx context.Context, zctx *zed.Context, filename string, ordinal int, zr zio.Reader) (*peeker, error) { @@ -32,7 +31,7 @@ func newPeeker(ctx context.Context, zctx *zed.Context, filename string, ordinal f.CloseAndRemove() return nil, err } - return &peeker{f, first, ordinal, zed.NewArena()}, nil + return &peeker{f, first, ordinal}, nil } // read is like Read but returns eof at the last record so a MergeSort can @@ -40,8 +39,7 @@ func newPeeker(ctx context.Context, zctx *zed.Context, filename string, ordinal func (p *peeker) read() (*zed.Value, bool, error) { rec := p.nextRecord if rec != nil { - p.arena.Reset() - rec = rec.Copy(p.arena).Ptr() + rec = rec.Copy().Ptr() } var err error p.nextRecord, err = p.Read() diff --git a/runtime/sam/op/switcher/switch.go b/runtime/sam/op/switcher/switch.go index 690a181376..275d5a65f4 100644 --- a/runtime/sam/op/switcher/switch.go +++ b/runtime/sam/op/switcher/switch.go @@ -11,7 +11,6 @@ import ( type Selector struct { *op.Router expr.Resetter - zctx *zed.Context cases []*switchCase } @@ -28,7 +27,6 @@ func New(rctx *runtime.Context, parent zbuf.Puller, resetter expr.Resetter) *Sel s := &Selector{ Router: router, Resetter: resetter, - zctx: rctx.Zctx, } router.Link(s) return s @@ -41,12 +39,11 @@ func (s *Selector) AddCase(f expr.Evaluator) zbuf.Puller { } func (s *Selector) Forward(router *op.Router, batch zbuf.Batch) bool { - arena := zed.NewArena() - defer arena.Unref() - ectx := expr.NewContextWithVars(arena, batch.Vars()) - for _, this := range batch.Values() { + vals := batch.Values() + for i := range vals { + this := vals[i] for _, c := range s.cases { - val := c.filter.Eval(ectx, this) + val := c.filter.Eval(batch, this) if val.IsMissing() { continue } @@ -69,7 +66,11 @@ func (s *Selector) Forward(router *op.Router, batch zbuf.Batch) bool { // ref the batch for each outgoing new batch. for _, c := range s.cases { if len(c.vals) > 0 { - out := zbuf.NewBatch(arena, c.vals, batch, batch.Vars()) + // XXX The new slice should come from the + // outgoing batch so we don't send these slices + // through GC. + batch.Ref() + out := zbuf.NewBatch(batch, c.vals) c.vals = nil if ok := router.Send(c.route, out, nil); !ok { return false diff --git a/runtime/sam/op/tail/tail.go b/runtime/sam/op/tail/tail.go index 3c6146df7c..7dd1eb26bf 100644 --- a/runtime/sam/op/tail/tail.go +++ b/runtime/sam/op/tail/tail.go @@ -75,7 +75,7 @@ func (o *Op) tail() ([]zbuf.Batch, error) { if n > o.limit { // We have too many values so remove some from batches[0]. vals := batches[0].Values()[n-o.limit:] - batches[0] = zbuf.WrapBatch(batches[0], vals) + batches[0] = zbuf.NewBatch(batches[0], vals) } return batches, nil } diff --git a/runtime/sam/op/top/top.go b/runtime/sam/op/top/top.go index c7dae92f14..c8de058987 100644 --- a/runtime/sam/op/top/top.go +++ b/runtime/sam/op/top/top.go @@ -17,13 +17,12 @@ const defaultTopLimit = 100 // the top N of the sort. // - It has a hidden option (FlushEvery) to sort and emit on every batch. type Op struct { - zctx *zed.Context parent zbuf.Puller + zctx *zed.Context limit int fields []expr.Evaluator flushEvery bool resetter expr.Resetter - batches map[zed.Value]zbuf.Batch records *expr.RecordSlice compare expr.CompareFn } @@ -33,13 +32,11 @@ func New(zctx *zed.Context, parent zbuf.Puller, limit int, fields []expr.Evaluat limit = defaultTopLimit } return &Op{ - zctx: zctx, parent: parent, limit: limit, fields: fields, flushEvery: flushEvery, resetter: resetter, - batches: make(map[zed.Value]zbuf.Batch), } } @@ -55,7 +52,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { } vals := batch.Values() for i := range vals { - o.consume(batch, vals[i]) + o.consume(vals[i]) } batch.Unref() if o.flushEvery { @@ -64,7 +61,7 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { } } -func (o *Op) consume(batch zbuf.Batch, rec zed.Value) { +func (o *Op) consume(rec zed.Value) { if o.fields == nil { fld := sort.GuessSortKey(rec) accessor := expr.NewDottedExpr(o.zctx, fld) @@ -76,17 +73,10 @@ func (o *Op) consume(batch zbuf.Batch, rec zed.Value) { heap.Init(o.records) } if o.records.Len() < o.limit || o.compare(o.records.Index(0), rec) < 0 { - heap.Push(o.records, rec) - if _, ok := rec.Arena(); ok { - batch.Ref() - o.batches[rec] = batch - } + heap.Push(o.records, rec.Copy()) } if o.records.Len() > o.limit { - val := heap.Pop(o.records).(zed.Value) - if batch, ok := o.batches[val]; ok { - batch.Unref() - } + heap.Pop(o.records) } } @@ -94,13 +84,11 @@ func (o *Op) sorted() zbuf.Batch { if o.records == nil { return nil } - arena := zed.NewArena() - defer arena.Unref() out := make([]zed.Value, o.records.Len()) for i := o.records.Len() - 1; i >= 0; i-- { - out[i] = heap.Pop(o.records).(zed.Value).Copy(arena) + out[i] = heap.Pop(o.records).(zed.Value) } // clear records o.records = nil - return zbuf.NewArray(arena, out) + return zbuf.NewArray(out) } diff --git a/runtime/sam/op/traverse/eval.go b/runtime/sam/op/traverse/eval.go index 4336ccc30d..7d1729c234 100644 --- a/runtime/sam/op/traverse/eval.go +++ b/runtime/sam/op/traverse/eval.go @@ -42,10 +42,12 @@ func (e *Expr) SetExit(exit *Exit) { } func (e *Expr) Eval(ectx expr.Context, this zed.Value) zed.Value { + b := zbuf.NewArray([]zed.Value{this}) + b.SetVars(ectx.Vars()) select { - case e.batchCh <- zbuf.NewBatchWithVars(ectx.Arena(), []zed.Value{this}, ectx.Vars()): + case e.batchCh <- b: case <-e.ctx.Done(): - return e.zctx.NewError(ectx.Arena(), e.ctx.Err()) + return e.zctx.NewError(e.ctx.Err()) } out := e.out[:0] for { @@ -81,7 +83,7 @@ func (e *Expr) makeArray(ectx expr.Context, vals []zed.Value) zed.Value { return zed.Null } if len(vals) == 1 { - return vals[0].Copy(ectx.Arena()) + return vals[0] } typ := vals[0].Type() for _, val := range vals[1:] { @@ -93,7 +95,7 @@ func (e *Expr) makeArray(ectx expr.Context, vals []zed.Value) zed.Value { for _, val := range vals { b.Append(val.Bytes()) } - return ectx.Arena().New(e.zctx.LookupTypeArray(typ), b.Bytes()) + return zed.NewValue(e.zctx.LookupTypeArray(typ), b.Bytes()) } func (e *Expr) makeUnionArray(ectx expr.Context, vals []zed.Value) zed.Value { @@ -110,7 +112,7 @@ func (e *Expr) makeUnionArray(ectx expr.Context, vals []zed.Value) zed.Value { for _, val := range vals { zed.BuildUnion(&b, union.TagOf(val.Type()), val.Bytes()) } - return ectx.Arena().New(e.zctx.LookupTypeArray(union), b.Bytes()) + return zed.NewValue(e.zctx.LookupTypeArray(union), b.Bytes()) } func (e *Expr) Pull(done bool) (zbuf.Batch, error) { diff --git a/runtime/sam/op/traverse/over.go b/runtime/sam/op/traverse/over.go index 864cc0c70f..dcbc9cfafc 100644 --- a/runtime/sam/op/traverse/over.go +++ b/runtime/sam/op/traverse/over.go @@ -31,7 +31,7 @@ func NewOver(rctx *runtime.Context, parent zbuf.Puller, exprs []expr.Evaluator, } func (o *Over) AddScope(ctx context.Context, names []string, exprs []expr.Evaluator) *Scope { - scope := newScope(ctx, o.zctx, o, names, exprs) + scope := newScope(ctx, o, names, exprs) o.enter = scope.enter return scope } @@ -54,11 +54,11 @@ func (o *Over) Pull(done bool) (zbuf.Batch, error) { } this := o.outer[0] o.outer = o.outer[1:] - batch := o.batch + ectx := o.batch if o.enter != nil { - batch = o.enter.addLocals(batch, this) + ectx = o.enter.addLocals(ectx, this) } - innerBatch := o.over(batch, this) + innerBatch := o.over(ectx, this) if len(o.outer) == 0 { o.batch.Unref() } @@ -69,34 +69,34 @@ func (o *Over) Pull(done bool) (zbuf.Batch, error) { } func (o *Over) over(batch zbuf.Batch, this zed.Value) zbuf.Batch { - arena := zed.NewArena() - defer arena.Unref() - ectx := expr.NewContextWithVars(arena, batch.Vars()) // Copy the vars into a new scope since downstream, nested subgraphs // can have concurrent operators. We can optimize these copies out // later depending on the nested subgraph. var vals []zed.Value for _, e := range o.exprs { - val := e.Eval(ectx, this) + val := e.Eval(batch, this) // Propagate errors but skip missing values. if !val.IsMissing() { - vals = appendOver(o.zctx, arena, vals, val) + vals = appendOver(o.zctx, vals, val) } } if len(vals) == 0 { return nil } - return zbuf.NewBatch(arena, vals, batch, batch.Vars()) + return zbuf.NewBatch(batch, vals) } -func appendOver(zctx *zed.Context, arena *zed.Arena, vals []zed.Value, val zed.Value) []zed.Value { - val = val.Under(arena) +func appendOver(zctx *zed.Context, vals []zed.Value, val zed.Value) []zed.Value { + val = val.Under() switch typ := zed.TypeUnder(val.Type()).(type) { case *zed.TypeArray, *zed.TypeSet: typ = zed.InnerType(typ) for it := val.Bytes().Iter(); !it.Done(); { - val := arena.New(typ, it.Next()).Under(arena) - vals = append(vals, val) + // XXX when we do proper expr.Context, we can allocate + // this copy through the batch. + val := zed.NewValue(typ, it.Next()) + val = val.Under() + vals = append(vals, val.Copy()) } return vals case *zed.TypeMap: @@ -106,7 +106,7 @@ func appendOver(zctx *zed.Context, arena *zed.Arena, vals []zed.Value, val zed.V }) for it := val.Bytes().Iter(); !it.Done(); { bytes := zcode.Append(zcode.Append(nil, it.Next()), it.Next()) - vals = append(vals, arena.New(rtyp, bytes)) + vals = append(vals, zed.NewValue(rtyp, bytes)) } return vals case *zed.TypeRecord: @@ -122,10 +122,10 @@ func appendOver(zctx *zed.Context, arena *zed.Arena, vals []zed.Value, val zed.V builder.Append(zed.EncodeString(field.Name)) builder.EndContainer() builder.Append(it.Next()) - vals = append(vals, arena.New(typ, builder.Bytes())) + vals = append(vals, zed.NewValue(typ, builder.Bytes()).Copy()) } return vals default: - return append(vals, val) + return append(vals, val.Copy()) } } diff --git a/runtime/sam/op/traverse/scope.go b/runtime/sam/op/traverse/scope.go index 33eb4f2ec8..55946ccacc 100644 --- a/runtime/sam/op/traverse/scope.go +++ b/runtime/sam/op/traverse/scope.go @@ -2,7 +2,6 @@ package traverse import ( "context" - "slices" "sync" "github.com/brimdata/zed" @@ -23,14 +22,14 @@ type Scope struct { subDoneCh chan struct{} } -func newScope(ctx context.Context, zctx *zed.Context, parent zbuf.Puller, names []string, exprs []expr.Evaluator) *Scope { +func newScope(ctx context.Context, parent zbuf.Puller, names []string, exprs []expr.Evaluator) *Scope { return &Scope{ ctx: ctx, parent: parent, // Buffered so we can send without blocking before we send EOS // to resultCh. parentEOSCh: make(chan struct{}, 1), - enter: NewEnter(zctx, names, exprs), + enter: NewEnter(names, exprs), resultCh: make(chan op.Result), exitDoneCh: make(chan struct{}), subDoneCh: make(chan struct{}), @@ -147,34 +146,30 @@ again: } type Enter struct { - zctx *zed.Context names []string exprs []expr.Evaluator } -func NewEnter(zctx *zed.Context, names []string, exprs []expr.Evaluator) *Enter { +func NewEnter(names []string, exprs []expr.Evaluator) *Enter { return &Enter{ - zctx: zctx, names: names, exprs: exprs, } } func (e *Enter) addLocals(batch zbuf.Batch, this zed.Value) zbuf.Batch { - arena := zed.NewArena() - defer arena.Unref() - vars := slices.Clone(batch.Vars()) - for _, ex := range e.exprs { + inner := newScopedBatch(batch, len(e.exprs)) + for _, expr := range e.exprs { // Note that we add a var to the frame on each Eval call // since subsequent expressions can refer to results from // previous expressions. Also, we push any val including // errors and missing as we want to propagate such conditions // into the sub-graph to ease debuging. In fact, the subgraph // can act accordingly in response to errors and missing. - val := ex.Eval(expr.NewContextWithVars(arena, vars), this) - vars = append(vars, val) + val := expr.Eval(inner, this) + inner.push(val.Copy()) } - return zbuf.NewBatch(arena, batch.Values(), batch, vars) + return inner } type Exit struct { @@ -255,6 +250,29 @@ type scope struct { var _ zbuf.Batch = (*scope)(nil) +func newScopedBatch(batch zbuf.Batch, nvar int) *scope { + vars := batch.Vars() + if len(vars) != 0 { + // XXX for now we just copy the slice. we can be + // more sophisticated later. + newvars := make([]zed.Value, len(vars), len(vars)+nvar) + copy(newvars, vars) + vars = newvars + } + return &scope{ + Batch: batch, + vars: vars, + } +} + +func (s *scope) Vars() []zed.Value { + return s.vars +} + +func (s *scope) push(val zed.Value) { + s.vars = append(s.vars, val) +} + type exitScope struct { zbuf.Batch vars []zed.Value diff --git a/runtime/sam/op/uniq/uniq.go b/runtime/sam/op/uniq/uniq.go index dffd411026..03f7eaeb17 100644 --- a/runtime/sam/op/uniq/uniq.go +++ b/runtime/sam/op/uniq/uniq.go @@ -15,9 +15,7 @@ type Op struct { builder zcode.Builder cflag bool count uint64 - last zed.Value - batch zbuf.Batch - arena *zed.Arena + last *zed.Value } func New(rctx *runtime.Context, parent zbuf.Puller, cflag bool) *Op { @@ -28,11 +26,8 @@ func New(rctx *runtime.Context, parent zbuf.Puller, cflag bool) *Op { } } -func (o *Op) wrap(t zed.Value) zed.Value { +func (o *Op) wrap(t *zed.Value) zed.Value { if o.cflag { - if o.arena == nil { - o.arena = zed.NewArena() - } o.builder.Reset() o.builder.Append(t.Bytes()) o.builder.Append(zed.EncodeUint(o.count)) @@ -40,14 +35,14 @@ func (o *Op) wrap(t zed.Value) zed.Value { zed.NewField("value", t.Type()), zed.NewField("count", zed.TypeUint64), }) - return o.arena.New(typ, o.builder.Bytes()) + return zed.NewValue(typ, o.builder.Bytes()).Copy() } - return t + return *t } -func (o *Op) appendUniq(out []zed.Value, t zed.Value) []zed.Value { +func (o *Op) appendUniq(out []zed.Value, t *zed.Value) []zed.Value { if o.count == 0 { - o.last = t + o.last = t.Copy().Ptr() o.count = 1 return out } else if bytes.Equal(t.Bytes(), o.last.Bytes()) { @@ -55,7 +50,7 @@ func (o *Op) appendUniq(out []zed.Value, t zed.Value) []zed.Value { return out } out = append(out, o.wrap(o.last)) - o.last = t + o.last = t.Copy().Ptr() o.count = 1 return out } @@ -69,24 +64,21 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { return nil, err } if batch == nil { - if o.count == 0 { + if o.last == nil { return nil, nil } t := o.wrap(o.last) - o.count = 0 - return zbuf.NewArray(o.arena, []zed.Value{t}), nil + o.last = nil + return zbuf.NewArray([]zed.Value{t}), nil } var out []zed.Value vals := batch.Values() for i := range vals { - out = o.appendUniq(out, vals[i]) - } - if o.batch != nil { - o.batch.Unref() + out = o.appendUniq(out, &vals[i]) } - o.batch = batch + batch.Unref() if len(out) > 0 { - return zbuf.NewBatch(o.arena, out, batch, batch.Vars()), nil + return zbuf.NewArray(out), nil } } } diff --git a/runtime/sam/op/yield/yield.go b/runtime/sam/op/yield/yield.go index af07525737..24da226f2b 100644 --- a/runtime/sam/op/yield/yield.go +++ b/runtime/sam/op/yield/yield.go @@ -27,25 +27,21 @@ func (o *Op) Pull(done bool) (zbuf.Batch, error) { o.resetter.Reset() return nil, err } - arena := zed.NewArena() - ectx := expr.NewContextWithVars(arena, batch.Vars()) vals := batch.Values() out := make([]zed.Value, 0, len(o.exprs)*len(vals)) for i := range vals { for _, e := range o.exprs { - val := e.Eval(ectx, vals[i]) + val := e.Eval(batch, vals[i]) if val.IsQuiet() { continue } - out = append(out, val) + out = append(out, val.Copy()) } } if len(out) > 0 { - defer arena.Unref() defer batch.Unref() - return zbuf.NewBatch(arena, out, batch, batch.Vars()), nil + return zbuf.NewBatch(batch, out), nil } - arena.Unref() batch.Unref() } } diff --git a/runtime/vam/expr/arith_test.go b/runtime/vam/expr/arith_test.go index 555c74ee09..4eac9e30e4 100644 --- a/runtime/vam/expr/arith_test.go +++ b/runtime/vam/expr/arith_test.go @@ -19,7 +19,7 @@ func TestArithOpsAndForms(t *testing.T) { rhsFlat := vector.NewInt(zed.TypeInt64, []int64{1, 1, 1}, nil) rhsDict := vector.NewDict(rhsFlat, []byte{0, 0, 0}, nil, nil) rhsView := vector.NewView([]uint32{0, 1, 2}, rhsFlat) - Const := vector.NewConst(nil, zed.NewInt64(1), 3, nil) + Const := vector.NewConst(zed.NewInt64(1), 3, nil) cases := []struct { op string diff --git a/runtime/vam/expr/arithfuncs.go b/runtime/vam/expr/arithfuncs.go index b1af1b7121..2c5ca45ebe 100644 --- a/runtime/vam/expr/arithfuncs.go +++ b/runtime/vam/expr/arithfuncs.go @@ -207,9 +207,8 @@ func arithAddIntConstView(lhs, rhs vector.Any) vector.Any { func arithAddIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - var arena *zed.Arena val := zed.NewInt64(lconst + rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithAddUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -412,9 +411,8 @@ func arithAddUintConstView(lhs, rhs vector.Any) vector.Any { func arithAddUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - var arena *zed.Arena val := zed.NewUint64(lconst + rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithAddFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -617,9 +615,8 @@ func arithAddFloatConstView(lhs, rhs vector.Any) vector.Any { func arithAddFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - var arena *zed.Arena val := zed.NewFloat64(lconst + rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithAddStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -822,10 +819,8 @@ func arithAddStringConstView(lhs, rhs vector.Any) vector.Any { func arithAddStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - var arena *zed.Arena - arena = zed.NewArena() - val := arena.NewString(lconst + rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + val := zed.NewString(lconst + rconst) + return vector.NewConst(val, lhs.Len(), nil) } func arithSubIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1028,9 +1023,8 @@ func arithSubIntConstView(lhs, rhs vector.Any) vector.Any { func arithSubIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - var arena *zed.Arena val := zed.NewInt64(lconst - rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithSubUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1233,9 +1227,8 @@ func arithSubUintConstView(lhs, rhs vector.Any) vector.Any { func arithSubUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - var arena *zed.Arena val := zed.NewUint64(lconst - rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithSubFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1438,9 +1431,8 @@ func arithSubFloatConstView(lhs, rhs vector.Any) vector.Any { func arithSubFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - var arena *zed.Arena val := zed.NewFloat64(lconst - rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithMulIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1643,9 +1635,8 @@ func arithMulIntConstView(lhs, rhs vector.Any) vector.Any { func arithMulIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - var arena *zed.Arena val := zed.NewInt64(lconst * rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithMulUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1848,9 +1839,8 @@ func arithMulUintConstView(lhs, rhs vector.Any) vector.Any { func arithMulUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - var arena *zed.Arena val := zed.NewUint64(lconst * rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithMulFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2053,9 +2043,8 @@ func arithMulFloatConstView(lhs, rhs vector.Any) vector.Any { func arithMulFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - var arena *zed.Arena val := zed.NewFloat64(lconst * rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithDivIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2258,9 +2247,8 @@ func arithDivIntConstView(lhs, rhs vector.Any) vector.Any { func arithDivIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - var arena *zed.Arena val := zed.NewInt64(lconst / rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithDivUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2463,9 +2451,8 @@ func arithDivUintConstView(lhs, rhs vector.Any) vector.Any { func arithDivUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - var arena *zed.Arena val := zed.NewUint64(lconst / rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithDivFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2668,9 +2655,8 @@ func arithDivFloatConstView(lhs, rhs vector.Any) vector.Any { func arithDivFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - var arena *zed.Arena val := zed.NewFloat64(lconst / rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithModIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2873,9 +2859,8 @@ func arithModIntConstView(lhs, rhs vector.Any) vector.Any { func arithModIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - var arena *zed.Arena val := zed.NewInt64(lconst % rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } func arithModUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -3078,9 +3063,8 @@ func arithModUintConstView(lhs, rhs vector.Any) vector.Any { func arithModUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - var arena *zed.Arena val := zed.NewUint64(lconst % rconst) - return vector.NewConst(arena, val, lhs.Len(), nil) + return vector.NewConst(val, lhs.Len(), nil) } var arithFuncs = map[int]func(vector.Any, vector.Any) vector.Any{ diff --git a/runtime/vam/expr/coerce.go b/runtime/vam/expr/coerce.go index abe43683bf..8156ac26bd 100644 --- a/runtime/vam/expr/coerce.go +++ b/runtime/vam/expr/coerce.go @@ -130,7 +130,7 @@ func promoteToSigned(val vector.Any) vector.Any { if !ok { panic("ToInt failed") } - return vector.NewConst(nil, zed.NewInt64(v), val.Len(), val.Nulls) + return vector.NewConst(zed.NewInt64(v), val.Len(), val.Nulls) case *vector.Dict: promoted := promoteToSigned(val.Any) return vector.NewDict(promoted, val.Index, val.Counts, val.Nulls) @@ -234,7 +234,7 @@ func intToFloat(val vector.Any) vector.Any { if !ok { panic("ToFloat failed") } - return vector.NewConst(nil, zed.NewFloat64(f), val.Len(), val.Nulls) + return vector.NewConst(zed.NewFloat64(f), val.Len(), val.Nulls) case *vector.Dict: return vector.NewDict(intToFloat(val.Any), val.Index, val.Counts, val.Nulls) case *vector.View: diff --git a/runtime/vam/expr/compare_test.go b/runtime/vam/expr/compare_test.go index 011e01731f..84840683b6 100644 --- a/runtime/vam/expr/compare_test.go +++ b/runtime/vam/expr/compare_test.go @@ -27,7 +27,7 @@ func TestCompareOpsAndForms(t *testing.T) { rhsFlat := vector.NewUint(zed.TypeUint64, []uint64{1, 1, 1}, nil) rhsDict := vector.NewDict(rhsFlat, []byte{0, 0, 0}, nil, nil) rhsView := vector.NewView([]uint32{0, 1, 2}, rhsFlat) - Const := vector.NewConst(nil, zed.NewUint64(1), 3, nil) + Const := vector.NewConst(zed.NewUint64(1), 3, nil) cases := []struct { op, expected, expectedForConstLHS string diff --git a/runtime/vam/expr/comparefuncs.go b/runtime/vam/expr/comparefuncs.go index 7cf37fb91b..cd3864e406 100644 --- a/runtime/vam/expr/comparefuncs.go +++ b/runtime/vam/expr/comparefuncs.go @@ -237,7 +237,7 @@ func compareEQIntConstView(lhs, rhs vector.Any) vector.Any { func compareEQIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - return vector.NewConst(nil, zed.NewBool(lconst == rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst == rconst), lhs.Len(), nil) } func compareEQUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -470,7 +470,7 @@ func compareEQUintConstView(lhs, rhs vector.Any) vector.Any { func compareEQUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - return vector.NewConst(nil, zed.NewBool(lconst == rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst == rconst), lhs.Len(), nil) } func compareEQFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -703,7 +703,7 @@ func compareEQFloatConstView(lhs, rhs vector.Any) vector.Any { func compareEQFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - return vector.NewConst(nil, zed.NewBool(lconst == rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst == rconst), lhs.Len(), nil) } func compareEQStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -936,7 +936,7 @@ func compareEQStringConstView(lhs, rhs vector.Any) vector.Any { func compareEQStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - return vector.NewConst(nil, zed.NewBool(lconst == rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst == rconst), lhs.Len(), nil) } func compareEQBytesFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1169,7 +1169,7 @@ func compareEQBytesConstView(lhs, rhs vector.Any) vector.Any { func compareEQBytesConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsBytes() rconst, _ := rhs.(*vector.Const).AsBytes() - return vector.NewConst(nil, zed.NewBool(string(lconst) == string(rconst)), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(string(lconst) == string(rconst)), lhs.Len(), nil) } func compareNEIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1402,7 +1402,7 @@ func compareNEIntConstView(lhs, rhs vector.Any) vector.Any { func compareNEIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - return vector.NewConst(nil, zed.NewBool(lconst != rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst != rconst), lhs.Len(), nil) } func compareNEUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1635,7 +1635,7 @@ func compareNEUintConstView(lhs, rhs vector.Any) vector.Any { func compareNEUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - return vector.NewConst(nil, zed.NewBool(lconst != rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst != rconst), lhs.Len(), nil) } func compareNEFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -1868,7 +1868,7 @@ func compareNEFloatConstView(lhs, rhs vector.Any) vector.Any { func compareNEFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - return vector.NewConst(nil, zed.NewBool(lconst != rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst != rconst), lhs.Len(), nil) } func compareNEStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2101,7 +2101,7 @@ func compareNEStringConstView(lhs, rhs vector.Any) vector.Any { func compareNEStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - return vector.NewConst(nil, zed.NewBool(lconst != rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst != rconst), lhs.Len(), nil) } func compareNEBytesFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2334,7 +2334,7 @@ func compareNEBytesConstView(lhs, rhs vector.Any) vector.Any { func compareNEBytesConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsBytes() rconst, _ := rhs.(*vector.Const).AsBytes() - return vector.NewConst(nil, zed.NewBool(string(lconst) != string(rconst)), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(string(lconst) != string(rconst)), lhs.Len(), nil) } func compareLTIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2567,7 +2567,7 @@ func compareLTIntConstView(lhs, rhs vector.Any) vector.Any { func compareLTIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - return vector.NewConst(nil, zed.NewBool(lconst < rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst < rconst), lhs.Len(), nil) } func compareLTUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -2800,7 +2800,7 @@ func compareLTUintConstView(lhs, rhs vector.Any) vector.Any { func compareLTUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - return vector.NewConst(nil, zed.NewBool(lconst < rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst < rconst), lhs.Len(), nil) } func compareLTFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -3033,7 +3033,7 @@ func compareLTFloatConstView(lhs, rhs vector.Any) vector.Any { func compareLTFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - return vector.NewConst(nil, zed.NewBool(lconst < rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst < rconst), lhs.Len(), nil) } func compareLTStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -3266,7 +3266,7 @@ func compareLTStringConstView(lhs, rhs vector.Any) vector.Any { func compareLTStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - return vector.NewConst(nil, zed.NewBool(lconst < rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst < rconst), lhs.Len(), nil) } func compareLTBytesFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -3499,7 +3499,7 @@ func compareLTBytesConstView(lhs, rhs vector.Any) vector.Any { func compareLTBytesConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsBytes() rconst, _ := rhs.(*vector.Const).AsBytes() - return vector.NewConst(nil, zed.NewBool(string(lconst) < string(rconst)), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(string(lconst) < string(rconst)), lhs.Len(), nil) } func compareLEIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -3732,7 +3732,7 @@ func compareLEIntConstView(lhs, rhs vector.Any) vector.Any { func compareLEIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - return vector.NewConst(nil, zed.NewBool(lconst <= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst <= rconst), lhs.Len(), nil) } func compareLEUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -3965,7 +3965,7 @@ func compareLEUintConstView(lhs, rhs vector.Any) vector.Any { func compareLEUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - return vector.NewConst(nil, zed.NewBool(lconst <= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst <= rconst), lhs.Len(), nil) } func compareLEFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -4198,7 +4198,7 @@ func compareLEFloatConstView(lhs, rhs vector.Any) vector.Any { func compareLEFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - return vector.NewConst(nil, zed.NewBool(lconst <= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst <= rconst), lhs.Len(), nil) } func compareLEStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -4431,7 +4431,7 @@ func compareLEStringConstView(lhs, rhs vector.Any) vector.Any { func compareLEStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - return vector.NewConst(nil, zed.NewBool(lconst <= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst <= rconst), lhs.Len(), nil) } func compareLEBytesFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -4664,7 +4664,7 @@ func compareLEBytesConstView(lhs, rhs vector.Any) vector.Any { func compareLEBytesConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsBytes() rconst, _ := rhs.(*vector.Const).AsBytes() - return vector.NewConst(nil, zed.NewBool(string(lconst) <= string(rconst)), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(string(lconst) <= string(rconst)), lhs.Len(), nil) } func compareGTIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -4897,7 +4897,7 @@ func compareGTIntConstView(lhs, rhs vector.Any) vector.Any { func compareGTIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - return vector.NewConst(nil, zed.NewBool(lconst > rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst > rconst), lhs.Len(), nil) } func compareGTUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -5130,7 +5130,7 @@ func compareGTUintConstView(lhs, rhs vector.Any) vector.Any { func compareGTUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - return vector.NewConst(nil, zed.NewBool(lconst > rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst > rconst), lhs.Len(), nil) } func compareGTFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -5363,7 +5363,7 @@ func compareGTFloatConstView(lhs, rhs vector.Any) vector.Any { func compareGTFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - return vector.NewConst(nil, zed.NewBool(lconst > rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst > rconst), lhs.Len(), nil) } func compareGTStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -5596,7 +5596,7 @@ func compareGTStringConstView(lhs, rhs vector.Any) vector.Any { func compareGTStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - return vector.NewConst(nil, zed.NewBool(lconst > rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst > rconst), lhs.Len(), nil) } func compareGTBytesFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -5829,7 +5829,7 @@ func compareGTBytesConstView(lhs, rhs vector.Any) vector.Any { func compareGTBytesConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsBytes() rconst, _ := rhs.(*vector.Const).AsBytes() - return vector.NewConst(nil, zed.NewBool(string(lconst) > string(rconst)), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(string(lconst) > string(rconst)), lhs.Len(), nil) } func compareGEIntFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -6062,7 +6062,7 @@ func compareGEIntConstView(lhs, rhs vector.Any) vector.Any { func compareGEIntConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsInt() rconst, _ := rhs.(*vector.Const).AsInt() - return vector.NewConst(nil, zed.NewBool(lconst >= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst >= rconst), lhs.Len(), nil) } func compareGEUintFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -6295,7 +6295,7 @@ func compareGEUintConstView(lhs, rhs vector.Any) vector.Any { func compareGEUintConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsUint() rconst, _ := rhs.(*vector.Const).AsUint() - return vector.NewConst(nil, zed.NewBool(lconst >= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst >= rconst), lhs.Len(), nil) } func compareGEFloatFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -6528,7 +6528,7 @@ func compareGEFloatConstView(lhs, rhs vector.Any) vector.Any { func compareGEFloatConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsFloat() rconst, _ := rhs.(*vector.Const).AsFloat() - return vector.NewConst(nil, zed.NewBool(lconst >= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst >= rconst), lhs.Len(), nil) } func compareGEStringFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -6761,7 +6761,7 @@ func compareGEStringConstView(lhs, rhs vector.Any) vector.Any { func compareGEStringConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsString() rconst, _ := rhs.(*vector.Const).AsString() - return vector.NewConst(nil, zed.NewBool(lconst >= rconst), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(lconst >= rconst), lhs.Len(), nil) } func compareGEBytesFlatFlat(lhs, rhs vector.Any) vector.Any { @@ -6994,7 +6994,7 @@ func compareGEBytesConstView(lhs, rhs vector.Any) vector.Any { func compareGEBytesConstConst(lhs, rhs vector.Any) vector.Any { lconst, _ := lhs.(*vector.Const).AsBytes() rconst, _ := rhs.(*vector.Const).AsBytes() - return vector.NewConst(nil, zed.NewBool(string(lconst) >= string(rconst)), lhs.Len(), nil) + return vector.NewConst(zed.NewBool(string(lconst) >= string(rconst)), lhs.Len(), nil) } var compareFuncs = map[int]func(vector.Any, vector.Any) vector.Any{ diff --git a/runtime/vam/expr/function/len.go b/runtime/vam/expr/function/len.go index 04492ca877..afa6a684e1 100644 --- a/runtime/vam/expr/function/len.go +++ b/runtime/vam/expr/function/len.go @@ -16,10 +16,10 @@ func (l *Len) Call(args ...vector.Any) vector.Any { out := vector.NewIntEmpty(zed.TypeInt64, val.Len(), nil) switch typ := val.Type().(type) { case *zed.TypeOfNull: - return vector.NewConst(nil, zed.NewInt64(0), val.Len(), nil) + return vector.NewConst(zed.NewInt64(0), val.Len(), nil) case *zed.TypeRecord: length := int64(len(typ.Fields)) - return vector.NewConst(nil, zed.NewInt64(length), val.Len(), nil) + return vector.NewConst(zed.NewInt64(length), val.Len(), nil) case *zed.TypeArray, *zed.TypeSet, *zed.TypeMap: for i := uint32(0); i < val.Len(); i++ { start, end, _ := vector.ContainerOffset(val, i) diff --git a/runtime/vam/expr/genarithfuncs.go b/runtime/vam/expr/genarithfuncs.go index 6d57c16226..12358bebae 100644 --- a/runtime/vam/expr/genarithfuncs.go +++ b/runtime/vam/expr/genarithfuncs.go @@ -72,14 +72,12 @@ func genFunc(name, op, typ string, lhs, rhs vector.Form) string { s += genVarInit("l", typ, lhs) s += genVarInit("r", typ, rhs) if lhs == vector.FormConst && rhs == vector.FormConst { - s += "var arena *zed.Arena\n" if typ == "String" { - s += "arena = zed.NewArena()\n" - s += fmt.Sprintf("val := arena.NewString(lconst %s rconst)\n", op) + s += fmt.Sprintf("val := zed.NewString(lconst %s rconst)\n", op) } else { s += fmt.Sprintf("val := zed.New%s64(lconst %s rconst)\n", typ, op) } - s += "return vector.NewConst(arena, val, lhs.Len(), nil)\n" + s += "return vector.NewConst(val, lhs.Len(), nil)\n" } else { s += "n := lhs.Len()\n" if typ == "String" { diff --git a/runtime/vam/expr/gencomparefuncs.go b/runtime/vam/expr/gencomparefuncs.go index 14e29e3dc9..b953f89563 100644 --- a/runtime/vam/expr/gencomparefuncs.go +++ b/runtime/vam/expr/gencomparefuncs.go @@ -75,7 +75,7 @@ func genFunc(name, op, typ string, lhs, rhs vector.Form) string { rexpr = "string(" + rexpr + ")" } if lhs == vector.FormConst && rhs == vector.FormConst { - s += fmt.Sprintf("return vector.NewConst(nil, zed.NewBool(%s %s %s), lhs.Len(), nil)\n", lexpr, op, rexpr) + s += fmt.Sprintf("return vector.NewConst(zed.NewBool(%s %s %s), lhs.Len(), nil)\n", lexpr, op, rexpr) } else { s += "n := lhs.Len()\n" s += "out := vector.NewBoolEmpty(n, nil)\n" diff --git a/runtime/vam/expr/literal.go b/runtime/vam/expr/literal.go index 609e2504d8..8bcde75b6e 100644 --- a/runtime/vam/expr/literal.go +++ b/runtime/vam/expr/literal.go @@ -6,16 +6,15 @@ import ( ) type Literal struct { - arena *zed.Arena - val zed.Value + val zed.Value } var _ Evaluator = (*Literal)(nil) -func NewLiteral(arena *zed.Arena, val zed.Value) *Literal { - return &Literal{arena, val} +func NewLiteral(val zed.Value) *Literal { + return &Literal{val: val} } func (l Literal) Eval(val vector.Any) vector.Any { - return vector.NewConst(l.arena, l.val, val.Len(), nil) + return vector.NewConst(l.val, val.Len(), nil) } diff --git a/runtime/vam/materialize.go b/runtime/vam/materialize.go index f1679db8b6..32a399e947 100644 --- a/runtime/vam/materialize.go +++ b/runtime/vam/materialize.go @@ -31,8 +31,6 @@ func (m *Materializer) Pull(done bool) (zbuf.Batch, error) { if variant == nil { typ = vec.Type() } - arena := zed.NewArena() - defer arena.Unref() builder := zcode.NewBuilder() var vals []zed.Value n := vec.Len() @@ -41,9 +39,9 @@ func (m *Materializer) Pull(done bool) (zbuf.Batch, error) { if variant != nil { typ = variant.TypeOf(slot) } - val := arena.New(typ, bytes.Clone(builder.Bytes().Body())) + val := zed.NewValue(typ, bytes.Clone(builder.Bytes().Body())) vals = append(vals, val) builder.Reset() } - return zbuf.NewArray(arena, vals), nil + return zbuf.NewArray(vals), nil } diff --git a/runtime/vam/op/scan.go b/runtime/vam/op/scan.go index ff1d04cf55..48ce5b1996 100644 --- a/runtime/vam/op/scan.go +++ b/runtime/vam/op/scan.go @@ -36,7 +36,7 @@ func NewScanner(rctx *runtime.Context, cache *vcache.Cache, parent zbuf.Puller, return &Scanner{ cache: cache, rctx: rctx, - parent: newObjectPuller(rctx.Zctx, parent), + parent: newObjectPuller(parent), pruner: pruner, pool: pool, projection: vcache.NewProjection(paths), @@ -80,7 +80,7 @@ func (s *Scanner) run() { s.sendResult(nil, err) return } - object, err := s.cache.Fetch(s.rctx.Context, s.rctx.Zctx, meta.VectorURI(s.pool.DataPath), meta.ID) + object, err := s.cache.Fetch(s.rctx.Context, meta.VectorURI(s.pool.DataPath), meta.ID) if err != nil { s.sendResult(nil, err) return @@ -122,14 +122,14 @@ type result struct { } type objectPuller struct { - parent zbuf.Puller - zctx *zed.Context + parent zbuf.Puller + unmarshaler *zson.UnmarshalZNGContext } -func newObjectPuller(zctx *zed.Context, parent zbuf.Puller) *objectPuller { +func newObjectPuller(parent zbuf.Puller) *objectPuller { return &objectPuller{ - parent: parent, - zctx: zctx, + parent: parent, + unmarshaler: zson.NewZNGUnmarshaler(), } } @@ -151,11 +151,9 @@ func (p *objectPuller) Pull(done bool) (*data.Object, error) { if named.Name != "data.Object" { return nil, fmt.Errorf("system error: vam.objectPuller encountered unnamed object: %q", named.Name) } - arena := zed.NewArena() var meta data.Object - if err := zson.UnmarshalZNG(p.zctx, arena, vals[0], &meta); err != nil { - return nil, fmt.Errorf("system error: vam.objectPuller could not unmarshal value %q: %w", zson.String(vals[0]), err) + if err := p.unmarshaler.Unmarshal(vals[0], &meta); err != nil { + return nil, fmt.Errorf("system error: vam.objectPuller could not unmarshal value: %q", zson.String(vals[0])) } - meta.Arena = arena return &meta, nil } diff --git a/runtime/vam/op/searcher.go b/runtime/vam/op/searcher.go index 9a23e49960..dd097f3846 100644 --- a/runtime/vam/op/searcher.go +++ b/runtime/vam/op/searcher.go @@ -30,7 +30,7 @@ func NewSearcher(rctx *runtime.Context, cache *vcache.Cache, parent zbuf.Puller, return &Searcher{ cache: cache, filter: filter, - parent: newObjectPuller(rctx.Zctx, parent), + parent: newObjectPuller(parent), pool: pool, projection: vcache.NewProjection(project), rctx: rctx, @@ -62,7 +62,7 @@ func (s *Searcher) run() { s.sendResult(nil, nil, err) return } - object, err := s.cache.Fetch(s.rctx.Context, s.rctx.Zctx, meta.VectorURI(s.pool.DataPath), meta.ID) + object, err := s.cache.Fetch(s.rctx.Context, meta.VectorURI(s.pool.DataPath), meta.ID) if err != nil { s.sendResult(nil, nil, err) return diff --git a/runtime/vcache/cache.go b/runtime/vcache/cache.go index e7dc1eb1d5..49c8af5e9b 100644 --- a/runtime/vcache/cache.go +++ b/runtime/vcache/cache.go @@ -4,7 +4,6 @@ import ( "context" "sync" - "github.com/brimdata/zed" "github.com/brimdata/zed/pkg/storage" "github.com/segmentio/ksuid" ) @@ -48,7 +47,7 @@ func (c *Cache) unlock(id ksuid.KSUID) { c.mu.Unlock() } -func (c *Cache) Fetch(ctx context.Context, zctx *zed.Context, uri *storage.URI, id ksuid.KSUID) (*Object, error) { +func (c *Cache) Fetch(ctx context.Context, uri *storage.URI, id ksuid.KSUID) (*Object, error) { c.mu.Lock() object, ok := c.objects[id] c.mu.Unlock() @@ -63,7 +62,7 @@ func (c *Cache) Fetch(ctx context.Context, zctx *zed.Context, uri *storage.URI, if ok { return object, nil } - object, err := NewObject(ctx, zctx, c.engine, uri) + object, err := NewObject(ctx, c.engine, uri) if err != nil { return nil, err } diff --git a/runtime/vcache/loader.go b/runtime/vcache/loader.go index f36fbe1dcf..8924235cc7 100644 --- a/runtime/vcache/loader.go +++ b/runtime/vcache/loader.go @@ -85,7 +85,7 @@ func (l *loader) loadVector(g *errgroup.Group, paths Path, s shadow) { s.mu.Lock() vec := s.vec if vec == nil { - vec = vector.NewConst(s.arena, s.val, s.length(), s.nulls.flat) + vec = vector.NewConst(s.val, s.length(), s.nulls.flat) s.vec = vec } s.mu.Unlock() @@ -287,7 +287,7 @@ func (l *loader) loadVals(typ zed.Type, s *primitive, nulls *vector.Bool) (vecto offs[length] = off return vector.NewTypeValue(offs, bytes, nulls), nil case *zed.TypeOfNull: - return vector.NewConst(nil, zed.Null, s.length(), nil), nil + return vector.NewConst(zed.Null, s.length(), nil), nil } return nil, fmt.Errorf("internal error: vcache.loadPrimitive got unknown type %#v", typ) } @@ -414,7 +414,7 @@ func empty(typ zed.Type, length uint32, nulls *vector.Bool) vector.Any { case *zed.TypeOfType: return vector.NewTypeValue(make([]uint32, length+1), nil, nulls) case *zed.TypeOfNull: - return vector.NewConst(nil, zed.Null, length, nil) + return vector.NewConst(zed.Null, length, nil) default: panic(fmt.Sprintf("vcache.empty: unknown type encountered: %T", typ)) } diff --git a/runtime/vcache/object.go b/runtime/vcache/object.go index 7914a7d55a..b3f95579a8 100644 --- a/runtime/vcache/object.go +++ b/runtime/vcache/object.go @@ -24,7 +24,7 @@ type Object struct { // residing in storage. The VNG header and metadata section are read and // the metadata is deserialized so that vectors can be loaded into the cache // on demand only as needed and retained in memory for future use. -func NewObject(ctx context.Context, zctx *zed.Context, engine storage.Engine, uri *storage.URI) (*Object, error) { +func NewObject(ctx context.Context, engine storage.Engine, uri *storage.URI) (*Object, error) { // XXX currently we open a storage.Reader for every object and never close it. // We should either close after a timeout and reopen when needed or change the // storage API to have a more reasonable semantics around the Put/Get not leaving @@ -35,7 +35,7 @@ func NewObject(ctx context.Context, zctx *zed.Context, engine storage.Engine, ur if err != nil { return nil, err } - object, err := vng.NewObject(zctx, reader) + object, err := vng.NewObject(reader) if err != nil { return nil, err } diff --git a/runtime/vcache/shadow.go b/runtime/vcache/shadow.go index a0563a737a..3fdad45ce5 100644 --- a/runtime/vcache/shadow.go +++ b/runtime/vcache/shadow.go @@ -93,8 +93,7 @@ type primitive struct { type const_ struct { mu sync.Mutex count - arena *zed.Arena - val zed.Value //XXX map this value? XXX, maybe wrap a shadow vector?, which could + val zed.Value //XXX map this value? XXX, maybe wrap a shadow vector?, which could // have a named in it vec *vector.Const nulls nulls @@ -206,7 +205,6 @@ func newShadow(m vng.Metadata, n *vng.Nulls, nullsCnt uint32) shadow { case *vng.Const: return &const_{ count: count{m.Len(), nullsCnt}, - arena: m.Arena, val: m.Value, nulls: nulls{meta: n}, } diff --git a/service/client_test.go b/service/client_test.go index 3681af58d6..b9c2ea0433 100644 --- a/service/client_test.go +++ b/service/client_test.go @@ -51,12 +51,9 @@ func (c *testClient) TestPoolList() []pools.Config { r, err := c.Query(context.Background(), nil, "from :pools") require.NoError(c, err) defer r.Body.Close() - arena := zed.NewArena() - defer arena.Unref() - zctx := zed.NewContext() - zr := zngio.NewReader(zctx, r.Body) - defer zr.Close() var confs []pools.Config + zr := zngio.NewReader(zed.NewContext(), r.Body) + defer zr.Close() for { rec, err := zr.Read() require.NoError(c, err) @@ -64,7 +61,7 @@ func (c *testClient) TestPoolList() []pools.Config { return confs } var pool pools.Config - err = zson.UnmarshalZNG(zctx, arena, *rec, &pool) + err = zson.UnmarshalZNG(*rec, &pool) require.NoError(c, err) confs = append(confs, pool) } diff --git a/service/core.go b/service/core.go index 889c19b733..5eb886d602 100644 --- a/service/core.go +++ b/service/core.go @@ -12,7 +12,6 @@ import ( "sync" "time" - "github.com/brimdata/zed" "github.com/brimdata/zed/api" "github.com/brimdata/zed/compiler" "github.com/brimdata/zed/lake" @@ -221,23 +220,20 @@ func (c *Core) ServeHTTP(w http.ResponseWriter, r *http.Request) { } func (c *Core) publishEvent(w *ResponseWriter, name string, data interface{}) { - arena := zed.NewArena() marshaler := zson.NewZNGMarshaler() marshaler.Decorate(zson.StyleSimple) - zv, err := marshaler.Marshal(arena, data) + zv, err := marshaler.Marshal(data) if err != nil { w.Logger.Error("Error marshaling published event", zap.Error(err)) return } go func() { - ev := event{name: name, arena: arena, value: zv} + ev := event{name: name, value: zv} c.subscriptionsMu.RLock() for sub := range c.subscriptions { - arena.Ref() sub <- ev } c.subscriptionsMu.RUnlock() - arena.Unref() }() } diff --git a/service/eventstream.go b/service/eventstream.go index 43d3b38c19..8a46ee5878 100644 --- a/service/eventstream.go +++ b/service/eventstream.go @@ -12,7 +12,6 @@ import ( type event struct { name string - arena *zed.Arena value zed.Value } @@ -33,7 +32,6 @@ func (e *eventStreamWriter) writeEvent(ev event) error { if err := w.Close(); err != nil { return err } - ev.arena.Unref() _, err = fmt.Fprintf(e.body, "event: %s\ndata: %s\n\n", ev.name, &buf) return err } diff --git a/service/request.go b/service/request.go index 118d0a1a59..a568b6af28 100644 --- a/service/request.go +++ b/service/request.go @@ -38,9 +38,12 @@ type Request struct { func newRequest(w http.ResponseWriter, r *http.Request, c *Core) (*ResponseWriter, *Request, bool) { req := &Request{Request: r} req.Logger = c.logger.With(zap.String("request_id", req.ID())) + m := zson.NewZNGMarshaler() + m.Decorate(zson.StylePackage) res := &ResponseWriter{ ResponseWriter: w, Logger: req.Logger, + marshaler: m, request: req, } ss := strings.Split(r.Header.Get("Accept"), ",") @@ -212,11 +215,12 @@ func (r *Request) format(w *ResponseWriter, dflt string) (string, bool) { type ResponseWriter struct { http.ResponseWriter - Format string - Logger *zap.Logger - zw zio.WriteCloser - request *Request - written int32 + Format string + Logger *zap.Logger + zw zio.WriteCloser + marshaler *zson.MarshalZNGContext + request *Request + written int32 } func (w *ResponseWriter) ContentType() string { @@ -277,11 +281,7 @@ func (w *ResponseWriter) Error(err error) { } func (w *ResponseWriter) Marshal(body interface{}) bool { - arena := zed.NewArena() - defer arena.Unref() - m := zson.NewZNGMarshaler() - m.Decorate(zson.StylePackage) - rec, err := m.Marshal(arena, body) + rec, err := w.marshaler.Marshal(body) if err != nil { // XXX If status header has not been sent this should send error. w.Error(err) diff --git a/value.go b/value.go index efcb58602b..747c413c44 100644 --- a/value.go +++ b/value.go @@ -2,10 +2,10 @@ package zed import ( "bytes" - "encoding/binary" "errors" "fmt" "math" + "net/netip" "runtime/debug" "unsafe" @@ -20,137 +20,78 @@ var ( ) var ( - NullUint8 = Value{a: aReprPrimitiveNull | IDUint8} - NullUint16 = Value{a: aReprPrimitiveNull | IDUint16} - NullUint32 = Value{a: aReprPrimitiveNull | IDUint32} - NullUint64 = Value{a: aReprPrimitiveNull | IDUint64} - NullInt8 = Value{a: aReprPrimitiveNull | IDInt8} - NullInt16 = Value{a: aReprPrimitiveNull | IDInt16} - NullInt32 = Value{a: aReprPrimitiveNull | IDInt32} - NullInt64 = Value{a: aReprPrimitiveNull | IDInt64} - NullDuration = Value{a: aReprPrimitiveNull | IDDuration} - NullTime = Value{a: aReprPrimitiveNull | IDTime} - NullFloat16 = Value{a: aReprPrimitiveNull | IDFloat16} - NullFloat32 = Value{a: aReprPrimitiveNull | IDFloat32} - NullFloat64 = Value{a: aReprPrimitiveNull | IDFloat64} - NullBool = Value{a: aReprPrimitiveNull | IDBool} - NullBytes = Value{a: aReprPrimitiveNull | IDBytes} - NullString = Value{a: aReprPrimitiveNull | IDString} - NullIP = Value{a: aReprPrimitiveNull | IDIP} - NullNet = Value{a: aReprPrimitiveNull | IDNet} - NullType = Value{a: aReprPrimitiveNull | IDType} - Null = Value{a: aReprPrimitiveNull | IDNull} + NullUint8 = Value{typ: TypeUint8} + NullUint16 = Value{typ: TypeUint16} + NullUint32 = Value{typ: TypeUint32} + NullUint64 = Value{typ: TypeUint64} + NullInt8 = Value{typ: TypeInt8} + NullInt16 = Value{typ: TypeInt16} + NullInt32 = Value{typ: TypeInt32} + NullInt64 = Value{typ: TypeInt64} + NullDuration = Value{typ: TypeDuration} + NullTime = Value{typ: TypeTime} + NullFloat16 = Value{typ: TypeFloat16} + NullFloat32 = Value{typ: TypeFloat32} + NullFloat64 = Value{typ: TypeFloat64} + NullBool = Value{typ: TypeBool} + NullBytes = Value{typ: TypeBytes} + NullString = Value{typ: TypeString} + NullIP = Value{typ: TypeIP} + NullNet = Value{typ: TypeNet} + NullType = Value{typ: TypeType} + Null = Value{typ: TypeNull} False = NewBool(false) True = NewBool(true) ) -const ( - aReprUnknown = 0 << 60 - aReprArena = 1 << 60 - aReprPrimitive = 2 << 60 - aReprPrimitiveNull = 3 << 60 - aReprSmallBytes = 4 << 60 - aReprsSmallString = 5 << 60 - aReprMask = 0xf << 60 - - aLengthMask = 0xf << 56 - aPrimitiveTypeIDMask = 0xff - - dStorageUnknown = 0 << 61 - dStorageSlices = 1 << 61 - dStorageBuffer = 2 << 61 - dStorageNull = 3 << 61 - dStorageValues = 4 << 61 - dStorageMask = 0x7 << 61 -) +type Allocator interface{} -// Value is a Zed value. type Value struct { - a uint64 - d uint64 -} - -// Arena returns v's Arena. If v does not belong to an Arena, it returns (nil, -// false). -func (v Value) Arena() (*Arena, bool) { - if v.a&aReprMask != aReprArena { - return nil, false - } - return (*Arena)(unsafe.Pointer(uintptr(v.a &^ aReprMask))), true -} - -func (v Value) arena() *Arena { - arena, ok := v.Arena() - if !ok { - panic(v) - } - return arena -} - -// IsZero returns whether v is the zero value (and therefore uninitialized). -func (v Value) IsZero() bool { - return v.a == 0 + typ Type + // If base == &nativeBase, len holds this Value's native representation. + // Otherwise, unsafe.Slice(base, len) holds its ZNG representation. + base *byte + len uint64 } func (v Value) Ptr() *Value { return &v } -func (v Value) Type() Type { - switch v.a & aReprMask { - case aReprArena: - return v.arena().type_(v.d) - case aReprPrimitive, aReprPrimitiveNull: - return idToType[v.a&aPrimitiveTypeIDMask] - case aReprSmallBytes: - return TypeBytes - case aReprsSmallString: - return TypeString - } - panic(v) -} - -var idToType = [...]Type{ - IDUint8: TypeUint8, - IDUint16: TypeUint16, - IDUint32: TypeUint32, - IDUint64: TypeUint64, - IDInt8: TypeInt8, - IDInt16: TypeInt16, - IDInt32: TypeInt32, - IDInt64: TypeInt64, - IDDuration: TypeDuration, - IDTime: TypeTime, - IDFloat16: TypeFloat16, - IDFloat32: TypeFloat32, - IDFloat64: TypeFloat64, - IDBool: TypeBool, - IDBytes: TypeBytes, - IDString: TypeString, - IDIP: TypeIP, - IDNet: TypeNet, - IDType: TypeType, - IDNull: TypeNull, -} - -func NewUint(t Type, x uint64) Value { return Value{uint64(aReprPrimitive | t.ID()), x} } -func NewUint8(u uint8) Value { return NewUint(TypeUint8, uint64(u)) } -func NewUint16(u uint16) Value { return NewUint(TypeUint16, uint64(u)) } -func NewUint32(u uint32) Value { return NewUint(TypeUint32, uint64(u)) } -func NewUint64(u uint64) Value { return NewUint(TypeUint64, u) } -func NewInt(t Type, x int64) Value { return Value{uint64(aReprPrimitive | t.ID()), uint64(x)} } -func NewInt8(i int8) Value { return NewInt(TypeInt8, int64(i)) } -func NewInt16(i int16) Value { return NewInt(TypeInt16, int64(i)) } -func NewInt32(i int32) Value { return NewInt(TypeInt32, int64(i)) } -func NewInt64(i int64) Value { return NewInt(TypeInt64, i) } -func NewDuration(d nano.Duration) Value { return NewInt(TypeDuration, int64(d)) } -func NewTime(ts nano.Ts) Value { return NewInt(TypeTime, int64(ts)) } -func NewFloat(t Type, x float64) Value { - return Value{uint64(aReprPrimitive | t.ID()), uint64(math.Float64bits(x))} -} -func NewFloat16(f float32) Value { return NewFloat(TypeFloat16, float64(f)) } -func NewFloat32(f float32) Value { return NewFloat(TypeFloat32, float64(f)) } -func NewFloat64(f float64) Value { return NewFloat(TypeFloat64, f) } -func NewBool(x bool) Value { return Value{aReprPrimitive | IDBool, boolToUint64(x)} } +func (v Value) Type() Type { return v.typ } + +func NewValue(t Type, b zcode.Bytes) Value { return Value{t, unsafe.SliceData(b), uint64(len(b))} } +func (v Value) bytes() zcode.Bytes { return unsafe.Slice(v.base, v.len) } + +// nativeBase is the base address for all native Values, which are encoded with +// the base field set to this address and the len field set to the bits of the +// Value's native representation. +var nativeBase byte + +func newNativeValue(t Type, x uint64) Value { return Value{t, &nativeBase, x} } +func (v Value) native() (uint64, bool) { return v.len, v.base == &nativeBase } + +func NewUint(t Type, u uint64) Value { return newNativeValue(t, u) } +func NewUint8(u uint8) Value { return newNativeValue(TypeUint8, uint64(u)) } +func NewUint16(u uint16) Value { return newNativeValue(TypeUint16, uint64(u)) } +func NewUint32(u uint32) Value { return newNativeValue(TypeUint32, uint64(u)) } +func NewUint64(u uint64) Value { return newNativeValue(TypeUint64, u) } +func NewInt(t Type, i int64) Value { return newNativeValue(t, uint64(i)) } +func NewInt8(i int8) Value { return newNativeValue(TypeInt8, uint64(i)) } +func NewInt16(i int16) Value { return newNativeValue(TypeInt16, uint64(i)) } +func NewInt32(i int32) Value { return newNativeValue(TypeInt32, uint64(i)) } +func NewInt64(i int64) Value { return newNativeValue(TypeInt64, uint64(i)) } +func NewDuration(d nano.Duration) Value { return newNativeValue(TypeDuration, uint64(d)) } +func NewTime(ts nano.Ts) Value { return newNativeValue(TypeTime, uint64(ts)) } +func NewFloat(t Type, f float64) Value { return newNativeValue(t, math.Float64bits(f)) } +func NewFloat16(f float32) Value { return newNativeValue(TypeFloat16, math.Float64bits(float64(f))) } +func NewFloat32(f float32) Value { return newNativeValue(TypeFloat32, math.Float64bits(float64(f))) } +func NewFloat64(f float64) Value { return newNativeValue(TypeFloat64, math.Float64bits(f)) } +func NewBool(b bool) Value { return newNativeValue(TypeBool, boolToUint64(b)) } +func NewBytes(b []byte) Value { return NewValue(TypeBytes, b) } +func NewString(s string) Value { return Value{TypeString, nonNilUnsafeStringData(s), uint64(len(s))} } +func NewIP(a netip.Addr) Value { return NewValue(TypeIP, EncodeIP(a)) } +func NewNet(p netip.Prefix) Value { return NewValue(TypeNet, EncodeNet(p)) } +func NewTypeValue(t Type) Value { return NewValue(TypeNet, EncodeTypeValue(t)) } func boolToUint64(b bool) uint64 { if b { @@ -159,102 +100,82 @@ func boolToUint64(b bool) uint64 { return 0 } -func (v Value) typeID() int { - switch v.a & aReprMask { - case aReprArena: - return v.arena().type_(v.d).ID() - case aReprPrimitive, aReprPrimitiveNull: - return int(v.a & aPrimitiveTypeIDMask) - case aReprSmallBytes: - return IDBytes - case aReprsSmallString: - return IDString +// nonNilUsafeStringData is like unsafe.StringData but never returns nil. +func nonNilUnsafeStringData(s string) *byte { + if d := unsafe.StringData(s); d != nil { + return d } - panic(v) + return unsafe.SliceData([]byte{}) } // Uint returns v's underlying value. It panics if v's underlying type is not // TypeUint8, TypeUint16, TypeUint32, or TypeUint64. func (v Value) Uint() uint64 { - if !IsUnsigned(v.typeID()) { + if v.Type().ID() > IDUint64 { panic(fmt.Sprintf("zed.Value.Uint called on %T", v.Type())) } - if v.a&aReprMask == aReprPrimitive { - return v.d + if x, ok := v.native(); ok { + return x } - return DecodeUint(v.arena().bytes_(v.d)) + return DecodeUint(v.bytes()) } // Int returns v's underlying value. It panics if v's underlying type is not // TypeInt8, TypeInt16, TypeInt32, TypeInt64, TypeDuration, or TypeTime. func (v Value) Int() int64 { - if !IsSigned(v.typeID()) { + if !IsSigned(v.Type().ID()) { panic(fmt.Sprintf("zed.Value.Int called on %T", v.Type())) } - if v.a&aReprMask == aReprPrimitive { - return int64(v.d) + if x, ok := v.native(); ok { + return int64(x) } - return DecodeInt(v.arena().bytes_(v.d)) + return DecodeInt(v.bytes()) } // Float returns v's underlying value. It panics if v's underlying type is not // TypeFloat16, TypeFloat32, or TypeFloat64. func (v Value) Float() float64 { - if !IsFloat(v.typeID()) { - panic(fmt.Sprintf("zed.Value.Float called on %T", v.Type)) + if !IsFloat(v.Type().ID()) { + panic(fmt.Sprintf("zed.Value.Float called on %T", v.Type())) } - if v.a&aReprMask == aReprPrimitive { - return math.Float64frombits(v.d) + if x, ok := v.native(); ok { + return math.Float64frombits(x) } - return DecodeFloat(v.arena().bytes_(v.d)) + return DecodeFloat(v.bytes()) } // Bool returns v's underlying value. It panics if v's underlying type is not // TypeBool. func (v Value) Bool() bool { - if v.typeID() != IDBool { - panic(fmt.Sprintf("zed.Value.Bool called on %T", v.Type)) + if v.Type().ID() != IDBool { + panic(fmt.Sprintf("zed.Value.Bool called on %T", v.Type())) } - return v.asBool() -} - -func (v Value) asBool() bool { - if v.a&aReprMask == aReprPrimitive { - return v.d != 0 + if x, ok := v.native(); ok { + return x != 0 } - return DecodeBool(v.arena().bytes_(v.d)) + return DecodeBool(v.bytes()) } // Bytes returns v's ZNG representation. func (v Value) Bytes() zcode.Bytes { - switch v.a & aReprMask { - case aReprArena: - return v.arena().bytes_(v.d) - case aReprPrimitive: - switch v.a & aPrimitiveTypeIDMask { + if x, ok := v.native(); ok { + switch v.Type().ID() { case IDUint8, IDUint16, IDUint32, IDUint64: - return EncodeUint(v.d) + return EncodeUint(x) case IDInt8, IDInt16, IDInt32, IDInt64, IDDuration, IDTime: - return EncodeInt(int64(v.d)) + return EncodeInt(int64(x)) case IDFloat16: - return EncodeFloat16(float32(math.Float64frombits(v.d))) + return EncodeFloat16(float32(math.Float64frombits(x))) case IDFloat32: - return EncodeFloat32(float32(math.Float64frombits(v.d))) + return EncodeFloat32(float32(math.Float64frombits(x))) case IDFloat64: - return EncodeFloat64(math.Float64frombits(v.d)) + return EncodeFloat64(math.Float64frombits(x)) case IDBool: - return EncodeBool(v.d != 0) + return EncodeBool(x != 0) } - case aReprPrimitiveNull: - return nil - case aReprSmallBytes, aReprsSmallString: - var b [16]byte - binary.BigEndian.PutUint64(b[:8], v.a) - binary.BigEndian.PutUint64(b[8:], v.d) - length := (v.a & aLengthMask) >> 56 - return b[1 : 1+length] + panic(v.Type()) } - panic(v) + return v.bytes() } func (v Value) IsContainer() bool { @@ -284,7 +205,7 @@ func (v Value) Iter() zcode.Iter { // element, and return its type and raw representation. Returns an // error if the passed-in element is not an array or if idx is // outside the array bounds. -func (v Value) ArrayIndex(arena *Arena, idx int64) (Value, error) { +func (v Value) ArrayIndex(idx int64) (Value, error) { vec, ok := v.Type().(*TypeArray) if !ok { return Null, ErrNotArray @@ -295,7 +216,7 @@ func (v Value) ArrayIndex(arena *Arena, idx int64) (Value, error) { for i, it := 0, v.Iter(); !it.Done(); i++ { bytes := it.Next() if i == int(idx) { - return arena.New(vec.Type, bytes), nil + return NewValue(vec.Type, bytes), nil } } return Null, ErrIndex @@ -303,14 +224,14 @@ func (v Value) ArrayIndex(arena *Arena, idx int64) (Value, error) { // Elements returns an array of Values for the given container type. // Returns an error if the element is not an array or set. -func (v Value) Elements(arena *Arena) ([]Value, error) { +func (v Value) Elements() ([]Value, error) { innerType := InnerType(v.Type()) if innerType == nil { return nil, ErrNotContainer } var elements []Value for it := v.Iter(); !it.Done(); { - elements = append(elements, arena.New(innerType, it.Next())) + elements = append(elements, NewValue(innerType, it.Next())) } return elements, nil } @@ -345,34 +266,26 @@ func (v Value) ContainerLength() (int, error) { // IsNull returns true if and only if v is a null value of any type. func (v Value) IsNull() bool { - return v.a&aReprMask == aReprPrimitiveNull || - v.a&aReprMask == aReprArena && v.d&dStorageMask == dStorageNull + return v.base == nil } -// Copy copies v to arena, returning a value that does not depend on any other -// arena. -func (v Value) Copy(arena *Arena) Value { - vArena, ok := v.Arena() - if !ok { - return v - } - switch v.d & dStorageMask { - case dStorageBuffer, dStorageSlices: - offset := len(arena.buf) - bytes := v.Bytes() - arena.buf = append(arena.buf, bytes...) - return arena.NewFromOffsetAndLength(v.Type(), offset, len(bytes)) - case dStorageNull: - return arena.New(v.Type(), nil) - case dStorageValues: - offset, length := vArena.offsetAndLength(v.d) - vals := make([]Value, 0, 32) - for _, val := range vArena.values[offset : offset+length] { - vals = append(vals, val.Copy(arena)) - } - return arena.NewFromValues(v.Type(), vals) +// Copy returns a copy of v that shares no storage. +func (v Value) Copy() Value { + if x, ok := v.native(); ok { + return newNativeValue(v.Type(), x) + } + return NewValue(v.Type(), bytes.Clone(v.bytes())) +} + +// CopyFrom copies from into v, reusing v's storage if possible. +func (v *Value) CopyFrom(from Value) { + if _, ok := from.native(); ok || from.IsNull() { + *v = from + } else if _, ok := v.native(); ok || v.IsNull() || v.len < from.len { + *v = NewValue(from.Type(), bytes.Clone(from.bytes())) + } else { + *v = NewValue(from.Type(), append(v.bytes()[:0], from.bytes()...)) } - panic(v) } func (v Value) IsString() bool { @@ -405,12 +318,14 @@ func (v Value) IsQuiet() bool { // Equal reports whether p and v have the same type and the same ZNG // representation. func (v Value) Equal(p Value) bool { - if v == p { - return true - } if v.Type() != p.Type() { return false } + if x, ok := v.native(); ok { + if y, ok := p.native(); ok { + return x == y + } + } return bytes.Equal(v.Bytes(), p.Bytes()) } @@ -439,10 +354,10 @@ func (r Value) Fields() []Field { return TypeRecordOf(r.Type()).Fields } -func (v *Value) DerefByColumn(arena *Arena, col int) *Value { +func (v *Value) DerefByColumn(col int) *Value { if v != nil { if bytes := v.nth(col); bytes != nil { - return arena.New(v.Fields()[col].Type, bytes).Ptr() + return NewValue(v.Fields()[col].Type, bytes).Ptr() } } return nil @@ -455,7 +370,7 @@ func (v Value) IndexOfField(field string) (int, bool) { return 0, false } -func (v *Value) Deref(arena *Arena, field string) *Value { +func (v *Value) Deref(field string) *Value { if v == nil { return nil } @@ -463,12 +378,12 @@ func (v *Value) Deref(arena *Arena, field string) *Value { if !ok { return nil } - return v.DerefByColumn(arena, i) + return v.DerefByColumn(i) } -func (v *Value) DerefPath(arena *Arena, path field.Path) *Value { +func (v *Value) DerefPath(path field.Path) *Value { for len(path) != 0 { - v = v.Deref(arena, path[0]) + v = v.Deref(path[0]) path = path[1:] } return v @@ -484,10 +399,10 @@ func (v *Value) AsString() string { // AsBool returns v's underlying value. It returns false if v is nil or v's // underlying type is not TypeBool. func (v *Value) AsBool() bool { - if v == nil || v.typeID() != IDBool { - return false + if v != nil && TypeUnder(v.Type()) == TypeBool { + return v.Bool() } - return v.asBool() + return false } func (v *Value) AsInt() int64 { @@ -518,23 +433,23 @@ func (v *Value) MissingAsNull() Value { // Under resolves named types and untags unions repeatedly, returning a value // guaranteed to have neither a named type nor a union type. -func (v Value) Under(arena *Arena) Value { +func (v Value) Under() Value { switch v.Type().(type) { case *TypeUnion, *TypeNamed: - return v.under(arena) + return v.under() } // This is the common case; make sure the compiler can inline it. return v } // under contains logic for Under that the compiler won't inline. -func (v Value) under(arena *Arena) Value { +func (v Value) under() Value { typ, bytes := v.Type(), v.Bytes() for { typ = TypeUnder(typ) union, ok := typ.(*TypeUnion) if !ok { - return arena.New(typ, bytes) + return NewValue(typ, bytes) } typ, bytes = union.Untag(bytes) } diff --git a/value_test.go b/value_test.go index 4c8c900c86..fea21ff3d4 100644 --- a/value_test.go +++ b/value_test.go @@ -5,15 +5,11 @@ import ( "github.com/brimdata/zed" "github.com/brimdata/zed/zcode" - "github.com/brimdata/zed/zson" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestNewStringNotNull(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - assert.NotNil(t, arena.NewString("").Bytes()) + assert.NotNil(t, zed.NewString("").Bytes()) } func BenchmarkValueUnder(b *testing.B) { @@ -21,27 +17,23 @@ func BenchmarkValueUnder(b *testing.B) { val := zed.Null b.ResetTimer() for i := 0; i < b.N; i++ { - val.Under(nil) + val.Under() } }) b.Run("named", func(b *testing.B) { - arena := zed.NewArena() - defer arena.Unref() typ, _ := zed.NewContext().LookupTypeNamed("name", zed.TypeNull) - val := arena.New(typ, nil) + val := zed.NewValue(typ, nil) b.ResetTimer() for i := 0; i < b.N; i++ { - val.Under(arena) + val.Under() } }) } func TestValueValidate(t *testing.T) { - zctx := zed.NewContext() - recType, err := zson.ParseType(zctx, "{f:|[string]|}") - require.NoError(t, err) - arena := zed.NewArena() - defer arena.Unref() + recType := zed.NewTypeRecord(0, []zed.Field{ + zed.NewField("f", zed.NewTypeSet(0, zed.TypeString)), + }) t.Run("set/error/duplicate-element", func(t *testing.T) { var b zcode.Builder b.BeginContainer() @@ -49,7 +41,7 @@ func TestValueValidate(t *testing.T) { b.Append([]byte("dup")) // Don't normalize. b.EndContainer() - val := arena.New(recType, b.Bytes()) + val := zed.NewValue(recType, b.Bytes()) assert.EqualError(t, val.Validate(), "invalid ZNG: duplicate set element") }) t.Run("set/error/unsorted-elements", func(t *testing.T) { @@ -60,7 +52,7 @@ func TestValueValidate(t *testing.T) { b.Append([]byte("b")) // Don't normalize. b.EndContainer() - val := arena.New(recType, b.Bytes()) + val := zed.NewValue(recType, b.Bytes()) assert.EqualError(t, val.Validate(), "invalid ZNG: set elements not sorted") }) t.Run("set/primitive-elements", func(t *testing.T) { @@ -72,7 +64,7 @@ func TestValueValidate(t *testing.T) { b.Append([]byte("a")) b.TransformContainer(zed.NormalizeSet) b.EndContainer() - val := arena.New(recType, b.Bytes()) + val := zed.NewValue(recType, b.Bytes()) assert.NoError(t, val.Validate()) }) t.Run("set/complex-elements", func(t *testing.T) { @@ -85,9 +77,13 @@ func TestValueValidate(t *testing.T) { } b.TransformContainer(zed.NormalizeSet) b.EndContainer() - typ, err := zson.ParseType(zctx, "{f:|[{g:string}]|}") - require.NoError(t, err) - val := arena.New(typ, b.Bytes()) - assert.NoError(t, val.Validate()) + r := zed.NewValue( + zed.NewTypeRecord(0, []zed.Field{ + zed.NewField("f", zed.NewTypeSet(0, zed.NewTypeRecord(0, []zed.Field{ + zed.NewField("g", zed.TypeString), + }))), + }), + b.Bytes()) + assert.NoError(t, r.Validate()) }) } diff --git a/vector/const.go b/vector/const.go index 04b5566fd3..150be5a0e1 100644 --- a/vector/const.go +++ b/vector/const.go @@ -7,7 +7,6 @@ import ( ) type Const struct { - arena *zed.Arena val zed.Value len uint32 Nulls *Bool @@ -15,8 +14,8 @@ type Const struct { var _ Any = (*Const)(nil) -func NewConst(arena *zed.Arena, val zed.Value, len uint32, nulls *Bool) *Const { - return &Const{arena: arena, val: val, len: len, Nulls: nulls} +func NewConst(val zed.Value, len uint32, nulls *Bool) *Const { + return &Const{val: val, len: len, Nulls: nulls} } func (c *Const) Type() zed.Type { diff --git a/vector/error.go b/vector/error.go index 5e835f61e1..1601b492b3 100644 --- a/vector/error.go +++ b/vector/error.go @@ -37,15 +37,13 @@ func (e *Error) Serialize(b *zcode.Builder, slot uint32) { } func NewStringError(zctx *zed.Context, msg string, len uint32) *Error { - arena := zed.NewArena() - vals := NewConst(arena, arena.NewString(msg), len, nil) + vals := NewConst(zed.NewString(msg), len, nil) return &Error{Typ: zctx.LookupTypeError(zed.TypeString), Vals: vals} } func NewMissing(zctx *zed.Context, len uint32) *Error { - arena := zed.NewArena() - missing := zctx.Missing(arena) - vals := NewConst(arena, missing, len, nil) + missing := zctx.Missing() + vals := NewConst(missing, len, nil) return &Error{Typ: missing.Type().(*zed.TypeError), Vals: vals} } @@ -54,8 +52,7 @@ func NewWrappedError(zctx *zed.Context, msg string, val Any) *Error { {Name: "message", Type: zed.TypeString}, {Name: "on", Type: val.Type()}, }) - arena := zed.NewArena() - sval := NewConst(arena, arena.NewString(msg), val.Len(), nil) + sval := NewConst(zed.NewString(msg), val.Len(), nil) rval := NewRecord(recType, []Any{sval, val}, val.Len(), nil) return &Error{Typ: zctx.LookupTypeError(recType), Vals: rval} } diff --git a/vector/vector_test.go b/vector/vector_test.go index 816d87c439..3361580899 100644 --- a/vector/vector_test.go +++ b/vector/vector_test.go @@ -2,10 +2,13 @@ package vector_test import ( "bytes" + "encoding/binary" + "math/rand" "testing" "github.com/brimdata/zed" "github.com/brimdata/zed/fuzz" + "github.com/brimdata/zed/vng" ) func FuzzQuery(f *testing.F) { @@ -16,10 +19,9 @@ func FuzzQuery(f *testing.F) { f.Fuzz(func(t *testing.T, b []byte) { bytesReader := bytes.NewReader(b) querySource := fuzz.GenAscii(bytesReader) - zctx := zed.NewContext() - types := fuzz.GenTypes(bytesReader, zctx, 3) - batch := fuzz.GenValues(bytesReader, zctx, types) - defer batch.Unref() + context := zed.NewContext() + types := fuzz.GenTypes(bytesReader, context, 3) + values := fuzz.GenValues(bytesReader, context, types) // Debug //for i := range values { @@ -28,15 +30,88 @@ func FuzzQuery(f *testing.F) { //} var zngBuf bytes.Buffer - fuzz.WriteZNG(t, batch, &zngBuf) + fuzz.WriteZNG(t, values, &zngBuf) resultZNG := fuzz.RunQueryZNG(t, &zngBuf, querySource) - defer resultZNG.Unref() var vngBuf bytes.Buffer - fuzz.WriteVNG(t, batch, &vngBuf) + fuzz.WriteVNG(t, values, &vngBuf) resultVNG := fuzz.RunQueryVNG(t, &vngBuf, querySource) - defer resultVNG.Unref() - fuzz.CompareValues(t, resultZNG.Values(), resultVNG.Values()) + fuzz.CompareValues(t, resultZNG, resultVNG) }) } + +const N = 10000000 + +func BenchmarkReadZng(b *testing.B) { + rand := rand.New(rand.NewSource(42)) + valuesIn := make([]zed.Value, N) + for i := range valuesIn { + valuesIn[i] = zed.NewInt64(rand.Int63n(N)) + } + var buf bytes.Buffer + fuzz.WriteZNG(b, valuesIn, &buf) + bs := buf.Bytes() + b.ResetTimer() + for i := 0; i < b.N; i++ { + valuesOut, err := fuzz.ReadZNG(bs) + if err != nil { + panic(err) + } + if zed.DecodeInt(valuesIn[N-1].Bytes()) != zed.DecodeInt(valuesOut[N-1].Bytes()) { + panic("oh no") + } + } +} + +func BenchmarkReadVng(b *testing.B) { + rand := rand.New(rand.NewSource(42)) + valuesIn := make([]zed.Value, N) + for i := range valuesIn { + valuesIn[i] = zed.NewValue(zed.TypeInt64, zed.EncodeInt(int64(rand.Intn(N)))) + } + var buf bytes.Buffer + fuzz.WriteVNG(b, valuesIn, &buf) + bs := buf.Bytes() + b.ResetTimer() + for i := 0; i < b.N; i++ { + bytesReader := bytes.NewReader(bs) + object, err := vng.NewObject(bytesReader) + if err != nil { + panic(err) + } + _ = object + // TODO Expose a cheap way to get values out of vectors. + //if intsIn[N-1] != intsOut[N-1] { + // panic("oh no") + //} + } +} + +func BenchmarkReadVarint(b *testing.B) { + rand := rand.New(rand.NewSource(42)) + intsIn := make([]int64, N) + for i := range intsIn { + intsIn[i] = int64(rand.Intn(N)) + } + var bs []byte + for _, int := range intsIn { + bs = binary.AppendVarint(bs, int) + } + b.ResetTimer() + for i := 0; i < b.N; i++ { + bs := bs + intsOut := make([]int64, N) + for i := range intsOut { + value, n := binary.Varint(bs) + if n <= 0 { + panic("oh no") + } + bs = bs[n:] + intsOut[i] = value + } + if intsIn[N-1] != intsOut[N-1] { + panic("oh no") + } + } +} diff --git a/vng/metadata.go b/vng/metadata.go index fb38af5836..e7c90f1f80 100644 --- a/vng/metadata.go +++ b/vng/metadata.go @@ -171,10 +171,6 @@ type Primitive struct { Min *zed.Value Max *zed.Value Count uint32 - - dictArena *zed.Arena - minArena *zed.Arena - maxArena *zed.Arena } func (p *Primitive) Type(zctx *zed.Context) zed.Type { @@ -200,7 +196,6 @@ func (n *Nulls) Len() uint32 { } type Const struct { - Arena *zed.Arena `zed:"-"` Value zed.Value Count uint32 } diff --git a/vng/object.go b/vng/object.go index 67071a7222..4025eaa6ee 100644 --- a/vng/object.go +++ b/vng/object.go @@ -35,26 +35,24 @@ import ( ) type Object struct { - readerAt io.ReaderAt - header Header - meta Metadata - metaArena *zed.Arena + readerAt io.ReaderAt + header Header + meta Metadata } -func NewObject(zctx *zed.Context, r io.ReaderAt) (*Object, error) { +func NewObject(r io.ReaderAt) (*Object, error) { hdr, err := ReadHeader(io.NewSectionReader(r, 0, HeaderSize)) if err != nil { return nil, err } - meta, arena, err := readMetadata(zctx, io.NewSectionReader(r, HeaderSize, int64(hdr.MetaSize))) + meta, err := readMetadata(io.NewSectionReader(r, HeaderSize, int64(hdr.MetaSize))) if err != nil { return nil, err } return &Object{ - readerAt: io.NewSectionReader(r, int64(HeaderSize+hdr.MetaSize), int64(hdr.DataSize)), - header: hdr, - meta: meta, - metaArena: arena, + readerAt: io.NewSectionReader(r, int64(HeaderSize+hdr.MetaSize), int64(hdr.DataSize)), + header: hdr, + meta: meta, }, nil } @@ -74,38 +72,29 @@ func (o *Object) DataReader() io.ReaderAt { } func (o *Object) NewReader(zctx *zed.Context) (zio.Reader, error) { - r, err := NewZedReader(zctx, o.meta, o.readerAt) - if err != nil { - return nil, err - } - return wrap{r, o.metaArena}, nil + return NewZedReader(zctx, o.meta, o.readerAt) } -type wrap struct { - zio.Reader - metaArena *zed.Arena -} - -func readMetadata(zctx *zed.Context, r io.Reader) (Metadata, *zed.Arena, error) { +func readMetadata(r io.Reader) (Metadata, error) { + zctx := zed.NewContext() zr := zngio.NewReader(zctx, r) defer zr.Close() val, err := zr.Read() if err != nil { - return nil, nil, err + return nil, err } - arena := zed.NewArena() u := zson.NewZNGUnmarshaler() - u.SetContext(zctx, arena) + u.SetContext(zctx) u.Bind(Template...) var meta Metadata if err := u.Unmarshal(*val, &meta); err != nil { - return nil, nil, err + return nil, err } // Read another val to make sure there is no extra stuff after the metadata. if extra, _ := zr.Read(); extra != nil { - return nil, nil, errors.New("corrupt VNG: metadata section has more than one Zed value") + return nil, errors.New("corrupt VNG: metadata section has more than one Zed value") } - return meta, arena, nil + return meta, nil } // XXX change this to single vector read diff --git a/vng/primitive.go b/vng/primitive.go index f15cebc785..5eb6551b28 100644 --- a/vng/primitive.go +++ b/vng/primitive.go @@ -3,7 +3,6 @@ package vng import ( "fmt" "io" - "slices" "sort" "github.com/brimdata/zed" @@ -26,10 +25,6 @@ type PrimitiveEncoder struct { min *zed.Value max *zed.Value count uint32 - - arena *zed.Arena - minArena *zed.Arena - maxArena *zed.Arena } func NewPrimitiveEncoder(typ zed.Type, useDict bool) *PrimitiveEncoder { @@ -45,10 +40,6 @@ func NewPrimitiveEncoder(typ zed.Type, useDict bool) *PrimitiveEncoder { typ: typ, dict: dict, cmp: expr.NewValueCompareFn(order.Asc, false), - - arena: zed.NewArena(), - minArena: zed.NewArena(), - maxArena: zed.NewArena(), } } @@ -62,15 +53,12 @@ func (p *PrimitiveEncoder) update(body zcode.Bytes) { if body == nil { panic("PrimitiveWriter should not be called with null") } - p.arena.Reset() - val := p.arena.New(p.typ, body) + val := zed.NewValue(p.typ, body) if p.min == nil || p.cmp(val, *p.min) < 0 { - p.minArena.Reset() - p.min = val.Copy(p.minArena).Ptr() + p.min = val.Copy().Ptr() } if p.max == nil || p.cmp(val, *p.max) > 0 { - p.maxArena.Reset() - p.max = val.Copy(p.maxArena).Ptr() + p.max = val.Copy().Ptr() } if p.dict != nil { p.dict[string(body)]++ @@ -98,8 +86,7 @@ func (p *PrimitiveEncoder) Encode(group *errgroup.Group) { } func (p *PrimitiveEncoder) makeDictVector() []byte { - dict, arena := p.makeDict() - defer arena.Unref() + dict := p.makeDict() pos := make(map[string]byte) for off, entry := range dict { if bytes := entry.Value.Bytes(); bytes != nil { @@ -133,17 +120,14 @@ func (p *PrimitiveEncoder) Const() *Const { bytes = []byte(b) } } - arena := zed.NewArena() return &Const{ - Arena: arena, - Value: arena.New(p.typ, bytes), + Value: zed.NewValue(p.typ, bytes), Count: p.count, } } func (p *PrimitiveEncoder) Metadata(off uint64) (uint64, Metadata) { var dict []DictEntry - var dictArena *zed.Arena if p.dict != nil { if cnt := len(p.dict); cnt != 0 { if cnt == 1 { @@ -153,7 +137,7 @@ func (p *PrimitiveEncoder) Metadata(off uint64) (uint64, Metadata) { p.out = nil return off, p.Const() } - dict, dictArena = p.makeDict() + dict = p.makeDict() } } loc := Segment{ @@ -164,15 +148,12 @@ func (p *PrimitiveEncoder) Metadata(off uint64) (uint64, Metadata) { } off += uint64(len(p.out)) return off, &Primitive{ - Typ: p.typ, - Location: loc, - Dict: dict, - Count: p.count, - Min: p.min, - Max: p.max, - dictArena: dictArena, - minArena: p.minArena, - maxArena: p.maxArena, + Typ: p.typ, + Location: loc, + Dict: dict, + Count: p.count, + Min: p.min, + Max: p.max, } } @@ -184,17 +165,16 @@ func (p *PrimitiveEncoder) Emit(w io.Writer) error { return err } -func (p *PrimitiveEncoder) makeDict() ([]DictEntry, *zed.Arena) { - arena := zed.NewArena() +func (p *PrimitiveEncoder) makeDict() []DictEntry { dict := make([]DictEntry, 0, len(p.dict)) for key, cnt := range p.dict { dict = append(dict, DictEntry{ - arena.New(p.typ, zcode.Bytes(key)), + zed.NewValue(p.typ, zcode.Bytes(key)), cnt, }) } - sortDict(dict, p.cmp) - return dict, arena + sortDict(dict, expr.NewValueCompareFn(order.Asc, false)) + return dict } func sortDict(entries []DictEntry, cmp expr.CompareFn) { @@ -251,8 +231,6 @@ type DictBuilder struct { dict []DictEntry selectors []byte off int - - dictArena *zed.Arena } var _ Builder = (*DictBuilder)(nil) @@ -263,8 +241,6 @@ func NewDictBuilder(primitive *Primitive, reader io.ReaderAt) *DictBuilder { reader: reader, loc: primitive.Location, dict: primitive.Dict, - - dictArena: primitive.dictArena, } } @@ -303,7 +279,7 @@ type ConstBuilder struct { var _ Builder = (*ConstBuilder)(nil) func NewConstBuilder(c *Const) *ConstBuilder { - return &ConstBuilder{Typ: c.Value.Type(), bytes: slices.Clone(c.Value.Bytes()), cnt: c.Count} + return &ConstBuilder{Typ: c.Value.Type(), bytes: c.Value.Bytes(), cnt: c.Count} } func (c *ConstBuilder) Build(b *zcode.Builder) error { diff --git a/vng/variant.go b/vng/variant.go index 963f28e7ee..5ddfcc63d2 100644 --- a/vng/variant.go +++ b/vng/variant.go @@ -11,7 +11,6 @@ import ( ) type VariantEncoder struct { - mapper *zed.Mapper tags *Int64Encoder values []Encoder which map[zed.Type]int @@ -20,11 +19,10 @@ type VariantEncoder struct { var _ zio.Writer = (*VariantEncoder)(nil) -func NewVariantEncoder(zctx *zed.Context) *VariantEncoder { +func NewVariantEncoder() *VariantEncoder { return &VariantEncoder{ - mapper: zed.NewMapper(zctx), - tags: NewInt64Encoder(), - which: make(map[zed.Type]int), + tags: NewInt64Encoder(), + which: make(map[zed.Type]int), } } @@ -33,10 +31,7 @@ func NewVariantEncoder(zctx *zed.Context) *VariantEncoder { // We track the types seen first-come, first-served and the // VNG metadata structure follows accordingly. func (v *VariantEncoder) Write(val zed.Value) error { - typ, err := v.mapper.Enter(val.Type()) - if err != nil { - return err - } + typ := val.Type() tag, ok := v.which[typ] if !ok { tag = len(v.values) @@ -97,7 +92,6 @@ type variantBuilder struct { tags *Int64Decoder values []Builder builder *zcode.Builder - arena *zed.Arena } func newVariantBuilder(zctx *zed.Context, variant *Variant, reader io.ReaderAt) (*variantBuilder, error) { @@ -116,7 +110,6 @@ func newVariantBuilder(zctx *zed.Context, variant *Variant, reader io.ReaderAt) tags: NewInt64Decoder(variant.Tags, reader), values: values, builder: zcode.NewBuilder(), - arena: zed.NewArena(), }, nil } @@ -136,8 +129,7 @@ func (v *variantBuilder) Read() (*zed.Value, error) { if err := v.values[tag].Build(b); err != nil { return nil, err } - v.arena.Reset() - return v.arena.New(v.types[tag], b.Bytes().Body()).Ptr(), nil + return zed.NewValue(v.types[tag], b.Bytes().Body()).Ptr(), nil } func NewZedReader(zctx *zed.Context, meta Metadata, r io.ReaderAt) (zio.Reader, error) { @@ -153,7 +145,6 @@ func NewZedReader(zctx *zed.Context, meta Metadata, r io.ReaderAt) (zio.Reader, values: values, builder: zcode.NewBuilder(), count: meta.Len(), - arena: zed.NewArena(), }, nil } @@ -162,7 +153,6 @@ type vectorBuilder struct { values Builder builder *zcode.Builder count uint32 - arena *zed.Arena } func (v *vectorBuilder) Read() (*zed.Value, error) { @@ -175,6 +165,5 @@ func (v *vectorBuilder) Read() (*zed.Value, error) { if err := v.values.Build(b); err != nil { return nil, err } - v.arena.Reset() - return v.arena.New(v.typ, b.Bytes().Body()).Ptr(), nil + return zed.NewValue(v.typ, b.Bytes().Body()).Ptr(), nil } diff --git a/vng/vng_test.go b/vng/vng_test.go index a1e465db7e..b6918e3235 100644 --- a/vng/vng_test.go +++ b/vng/vng_test.go @@ -7,7 +7,6 @@ import ( "github.com/brimdata/zed" "github.com/brimdata/zed/compiler/optimizer/demand" "github.com/brimdata/zed/fuzz" - "github.com/brimdata/zed/zbuf" "github.com/stretchr/testify/require" ) @@ -16,9 +15,8 @@ func FuzzVngRoundtripGen(f *testing.F) { bytesReader := bytes.NewReader(b) context := zed.NewContext() types := fuzz.GenTypes(bytesReader, context, 3) - batch := fuzz.GenValues(bytesReader, context, types) - defer batch.Unref() - roundtrip(t, batch) + values := fuzz.GenValues(bytesReader, context, types) + roundtrip(t, values) }) } @@ -32,10 +30,10 @@ func FuzzVngRoundtripBytes(f *testing.F) { }) } -func roundtrip(t *testing.T, batch zbuf.Batch) { +func roundtrip(t *testing.T, valuesIn []zed.Value) { var buf bytes.Buffer - fuzz.WriteVNG(t, batch, &buf) - batchOut, err := fuzz.ReadVNG(buf.Bytes(), demand.All()) + fuzz.WriteVNG(t, valuesIn, &buf) + valuesOut, err := fuzz.ReadVNG(buf.Bytes(), demand.All()) require.NoError(t, err) - fuzz.CompareValues(t, batch.Values(), batchOut.Values()) + fuzz.CompareValues(t, valuesIn, valuesOut) } diff --git a/vng/writer.go b/vng/writer.go index 0b9707c808..11d6f5c497 100644 --- a/vng/writer.go +++ b/vng/writer.go @@ -22,11 +22,10 @@ type Writer struct { var _ zio.Writer = (*Writer)(nil) func NewWriter(w io.WriteCloser) *Writer { - zctx := zed.NewContext() return &Writer{ - zctx: zctx, + zctx: zed.NewContext(), writer: w, - variant: NewVariantEncoder(zctx), + variant: NewVariantEncoder(), } } @@ -54,9 +53,7 @@ func (w *Writer) finalize() error { // First, we write the root segmap of the vector of integer type IDs. m := zson.NewZNGMarshalerWithContext(w.zctx) m.Decorate(zson.StyleSimple) - arena := zed.NewArena() - defer arena.Unref() - val, err := m.Marshal(arena, meta) + val, err := m.Marshal(meta) if err != nil { return fmt.Errorf("system error: could not marshal VNG metadata: %w", err) } diff --git a/vng/ztests/no-dict.yaml b/vng/ztests/no-dict.yaml index cf1a605590..c4a35edbb0 100644 --- a/vng/ztests/no-dict.yaml +++ b/vng/ztests/no-dict.yaml @@ -5,4 +5,4 @@ script: | outputs: - name: stdout data: | - null ([DictEntry={Value:{a:uint64,d:uint64},Count:uint32}]) + null ([DictEntry={Value:{typ:null,base:uint8,len:uint64},Count:uint32}]) diff --git a/zbuf/array.go b/zbuf/array.go index 6a2372f4ed..e72488be62 100644 --- a/zbuf/array.go +++ b/zbuf/array.go @@ -8,8 +8,8 @@ import ( // Array is a slice of of records that implements the Batch and // the Reader interfaces. type Array struct { - arena *zed.Arena values []zed.Value + vars []zed.Value } var _ Batch = (*Array)(nil) @@ -18,11 +18,8 @@ var _ zio.Writer = (*Array)(nil) // XXX this should take the frame arg too and the procs that create // new arrays need to propagate their frames downstream. -func NewArray(arena *zed.Arena, vals []zed.Value) *Array { - if arena != nil { - arena.Ref() - } - return &Array{arena, vals} +func NewArray(vals []zed.Value) *Array { + return &Array{values: vals} } func (a *Array) Ref() { @@ -33,31 +30,27 @@ func (a *Array) Unref() { // do nothing... let the GC reclaim it } -func (a *Array) Reset() { - if a.arena != nil { - a.arena.Reset() - } - a.values = a.values[:0] -} - func (a *Array) Values() []zed.Value { return a.values } +func (a *Array) Append(r zed.Value) { + a.values = append(a.values, r) +} + +func (a *Array) SetVars(vars []zed.Value) { + a.vars = vars +} + func (a *Array) Vars() []zed.Value { - return nil + return a.vars } func (a *Array) Write(r zed.Value) error { - if a.arena == nil { - a.arena = zed.NewArena() - } - a.values = append(a.values, r.Copy(a.arena)) + a.Append(r.Copy()) return nil } -func (*Array) Zctx() *zed.Arena { panic("zbuf.Array.Zctx") } - // Read returns removes the first element of the Array and returns it, // or it returns nil if the Array is empty. func (a *Array) Read() (*zed.Value, error) { diff --git a/zbuf/array_test.go b/zbuf/array_test.go new file mode 100644 index 0000000000..33022adf57 --- /dev/null +++ b/zbuf/array_test.go @@ -0,0 +1,16 @@ +package zbuf + +import ( + "testing" + + "github.com/brimdata/zed" + "github.com/stretchr/testify/require" +) + +func TestArrayWriteCopiesValueBytes(t *testing.T) { + var a Array + val := zed.NewBytes([]byte{0}) + a.Write(val) + copy(val.Bytes(), zed.EncodeBytes([]byte{1})) + require.Equal(t, zed.NewBytes([]byte{0}), a.Values()[0]) +} diff --git a/zbuf/batch.go b/zbuf/batch.go index c6ee225a6f..e256b6af73 100644 --- a/zbuf/batch.go +++ b/zbuf/batch.go @@ -1,6 +1,8 @@ package zbuf import ( + "slices" + "sync" "sync/atomic" "github.com/brimdata/zed" @@ -23,78 +25,20 @@ import ( // // Regardless of reference count or implementation, an unreachable Batch will // eventually be reclaimed by the garbage collector. - type Batch interface { Ref() Unref() Values() []zed.Value + // Vars accesses the variables reachable in the current scope. Vars() []zed.Value } -type batch struct { - refs int32 - arena *zed.Arena - vals []zed.Value - batch Batch - vars []zed.Value - batches []Batch - free func() -} - -func WrapBatch(b Batch, vals []zed.Value) Batch { - return NewBatch(nil, vals, b, b.Vars()) -} - -func NewBatchWithVars(arena *zed.Arena, vals []zed.Value, vars []zed.Value) Batch { - return NewBatch(arena, vals, nil, vars) -} - -func NewBatchWithVarsAndFree(arena *zed.Arena, vals []zed.Value, vars []zed.Value, free func()) Batch { - b := NewBatch(arena, vals, nil, vars) - b.(*batch).free = free - return b -} - -func NewBatch(arena *zed.Arena, vals []zed.Value, b Batch, vars []zed.Value) Batch { - if arena != nil { - arena.Ref() - } - if b != nil { - b.Ref() - } - return &batch{1, arena, vals, b, vars, nil, nil} -} - -func (b *batch) AddBatches(batches ...Batch) { - b.batches = append(b.batches, batches...) -} - -func (b *batch) Ref() { atomic.AddInt32(&b.refs, 1) } - -func (b *batch) Unref() { - if refs := atomic.AddInt32(&b.refs, -1); refs == 0 { - if b.arena != nil { - b.arena.Unref() - } - if b.batch != nil { - b.batch.Unref() - } - if b.free != nil { - b.free() - } - } else if refs < 0 { - panic("zbuf: negative batch reference count") - } -} - -func (b *batch) Values() []zed.Value { return b.vals } -func (b *batch) Vars() []zed.Value { return b.vars } - // WriteBatch writes the values in batch to zw. If an error occurs, WriteBatch // stops and returns the error. func WriteBatch(zw zio.Writer, batch Batch) error { - for _, val := range batch.Values() { - if err := zw.Write(val); err != nil { + vals := batch.Values() + for i := range vals { + if err := zw.Write(vals[i]); err != nil { return err } } @@ -114,12 +58,16 @@ type Puller interface { Pull(bool) (Batch, error) } +// PullerBatchBytes is the maximum number of bytes (in the zed.Value.Byte +// sense) per batch for a [Puller] created by [NewPuller]. +const PullerBatchBytes = 512 * 1024 + // PullerBatchValues is the maximum number of values per batch for a [Puller] // created by [NewPuller]. var PullerBatchValues = 100 // NewPuller returns a puller for zr that returns batches containing up to -// [PullerBatchValues] values. +// [PullerBatchBytes] bytes and [PullerBatchValues] values. func NewPuller(zr zio.Reader) Puller { return &puller{zr} } @@ -135,9 +83,7 @@ func (p *puller) Pull(done bool) (Batch, error) { if p.zr == nil { return nil, nil } - arena := zed.NewArena() - defer arena.Unref() - vals := make([]zed.Value, 0, PullerBatchValues) + batch := newPullerBatch() for { val, err := p.zr.Read() if err != nil { @@ -145,18 +91,75 @@ func (p *puller) Pull(done bool) (Batch, error) { } if val == nil { p.zr = nil - if len(vals) == 0 { + if len(batch.vals) == 0 { return nil, nil } - return NewBatch(arena, vals, nil, nil), nil + return batch, nil + } + if batch.appendVal(*val) { + return batch, nil + } + } +} + +type pullerBatch struct { + buf []byte + refs atomic.Int32 + vals []zed.Value +} + +var pullerBatchPool sync.Pool + +func newPullerBatch() *pullerBatch { + b, ok := pullerBatchPool.Get().(*pullerBatch) + if !ok { + b = &pullerBatch{ + buf: make([]byte, PullerBatchBytes), + vals: make([]zed.Value, PullerBatchValues), } - vals = append(vals, val.Copy(arena)) - if len(vals) >= PullerBatchValues { - return NewBatch(arena, vals, nil, nil), nil + } + b.buf = b.buf[:0] + b.refs.Store(1) + b.vals = b.vals[:0] + return b +} + +// appendVal appends a copy of val to b. appendVal returns true if b is full +// (i.e., b.buf is full, b.buf had insufficient space for val.Bytes, or b.val is +// full). appendVal never reallocates b.buf or b.vals. +func (b *pullerBatch) appendVal(val zed.Value) bool { + var bytes []byte + var bufFull bool + if !val.IsNull() { + if avail := cap(b.buf) - len(b.buf); avail >= len(val.Bytes()) { + // Append to b.buf since that won't reallocate. + start := len(b.buf) + b.buf = append(b.buf, val.Bytes()...) + bytes = b.buf[start:] + bufFull = avail == len(val.Bytes()) + } else { + // Copy since appending to b.buf would reallocate. + bytes = slices.Clone(val.Bytes()) + bufFull = true } } + b.vals = append(b.vals, zed.NewValue(val.Type(), bytes)) + return bufFull || len(b.vals) == cap(b.vals) +} + +func (b *pullerBatch) Ref() { b.refs.Add(1) } + +func (b *pullerBatch) Unref() { + if refs := b.refs.Add(-1); refs == 0 { + pullerBatchPool.Put(b) + } else if refs < 0 { + panic("zbuf: negative batch reference count") + } } +func (p *pullerBatch) Values() []zed.Value { return p.vals } +func (*pullerBatch) Vars() []zed.Value { return nil } + func CopyPuller(w zio.Writer, p Puller) error { for { b, err := p.Pull(false) @@ -198,3 +201,41 @@ func (r *pullerReader) Read() (*zed.Value, error) { r.vals = r.vals[1:] return val, nil } + +// XXX at some point the stacked scopes should not make copies of values +// but merely refer back to the value in the wrapped batch, and we should +// ref the wrapped batch then downstream entities will unref it, but how +// do we carry the var frame through... protocol needs to be that any new +// batch created by a proc needs to preserve the var frame... we don't +// do that right now and ref counting needs to account for the dependencies. +// procs like summarize and sort that unref their input batches merely need +// to copy the first frame (of each batch) and the contract is that the +// frame will not change between multiple batches within a single-EOS event. + +type batch struct { + Batch + vars []zed.Value +} + +func NewBatch(b Batch, vals []zed.Value) Batch { + return &batch{ + Batch: NewArray(vals), + vars: CopyVars(b), + } +} + +func (b *batch) Vars() []zed.Value { + return b.vars +} + +func CopyVars(b Batch) []zed.Value { + vars := b.Vars() + if len(vars) > 0 { + newvars := make([]zed.Value, len(vars)) + for k, v := range vars { + newvars[k] = v.Copy() + } + vars = newvars + } + return vars +} diff --git a/zbuf/merger.go b/zbuf/merger.go index 00bbc3b6e8..1ce0762df3 100644 --- a/zbuf/merger.go +++ b/zbuf/merger.go @@ -34,5 +34,5 @@ func NewComparatorNullsMax(zctx *zed.Context, sortKeys order.SortKeys) *expr.Com type valueAsBytes struct{} func (v *valueAsBytes) Eval(ectx expr.Context, val zed.Value) zed.Value { - return ectx.Arena().NewBytes(val.Bytes()) + return zed.NewBytes(val.Bytes()) } diff --git a/zbuf/scanner.go b/zbuf/scanner.go index 0594071d36..dd8ed3718b 100644 --- a/zbuf/scanner.go +++ b/zbuf/scanner.go @@ -99,7 +99,7 @@ func newScanner(ctx context.Context, r zio.Reader, filterExpr Filter) (Scanner, return nil, err } } - sc := &scanner{reader: r, filter: f, ctx: ctx, ectx: expr.NewContext(zed.NewArena())} + sc := &scanner{reader: r, filter: f, ctx: ctx, ectx: expr.NewContext()} sc.Puller = NewPuller(sc) return sc, nil } @@ -130,7 +130,6 @@ func (s *scanner) Read() (*zed.Value, error) { atomic.AddInt64(&s.progress.BytesRead, int64(len(this.Bytes()))) atomic.AddInt64(&s.progress.RecordsRead, 1) if s.filter != nil { - s.ectx.Arena().Reset() val := s.filter.Eval(s.ectx, *this) if !(val.Type() == zed.TypeBool && val.Bool()) { continue diff --git a/zio/arrowio/reader.go b/zio/arrowio/reader.go index ef782aa75e..1259ee4640 100644 --- a/zio/arrowio/reader.go +++ b/zio/arrowio/reader.go @@ -18,9 +18,8 @@ import ( // Reader is a zio.Reader for the Arrow IPC stream format. type Reader struct { - arena *zed.Arena - zctx *zed.Context - rr pqarrow.RecordReader + zctx *zed.Context + rr pqarrow.RecordReader typ zed.Type unionTagMappings map[string][]int @@ -49,7 +48,6 @@ func NewReaderFromRecordReader(zctx *zed.Context, rr pqarrow.RecordReader) (*Rea fields := slices.Clone(rr.Schema().Fields()) uniquifyFieldNames(fields) r := &Reader{ - arena: zed.NewArena(), zctx: zctx, rr: rr, unionTagMappings: map[string][]int{}, @@ -106,8 +104,7 @@ func (r *Reader) Read() (*zed.Value, error) { return nil, err } } - r.arena.Reset() - r.val = r.arena.New(r.typ, r.builder.Bytes()) + r.val = zed.NewValue(r.typ, r.builder.Bytes()) r.i++ if r.i >= int(r.rec.NumRows()) { r.rec.Release() diff --git a/zio/arrowio/writer.go b/zio/arrowio/writer.go index a98a5f2029..769c993e22 100644 --- a/zio/arrowio/writer.go +++ b/zio/arrowio/writer.go @@ -355,7 +355,7 @@ func (w *Writer) buildArrowValue(b array.Builder, typ zed.Type, bytes zcode.Byte } b.Append(s) case *zed.TypeError: - b.Append(zson.FormatTypeAndBytes(typ, bytes)) + b.Append(zson.FormatValue(zed.NewValue(typ, bytes))) default: panic(fmt.Sprintf("unexpected Zed type for StringBuilder: %s", zson.FormatType(typ))) } diff --git a/zio/csvio/reader.go b/zio/csvio/reader.go index 99d8b4c572..bd69167e15 100644 --- a/zio/csvio/reader.go +++ b/zio/csvio/reader.go @@ -13,7 +13,6 @@ import ( ) type Reader struct { - arena *zed.Arena reader *csv.Reader marshaler *zson.MarshalZNGContext strings bool @@ -49,7 +48,6 @@ func NewReader(zctx *zed.Context, r io.Reader, opts ReaderOpts) *Reader { } reader.ReuseRecord = true return &Reader{ - arena: zed.NewArena(), reader: reader, marshaler: zson.NewZNGMarshalerWithContext(zctx), } @@ -100,8 +98,7 @@ func (r *Reader) translate(fields []string) (zed.Value, error) { vals = append(vals, convertString(field)) } } - r.arena.Reset() - return r.marshaler.MarshalCustom(r.arena, r.hdr, vals) + return r.marshaler.MarshalCustom(r.hdr, vals) } func convertString(s string) interface{} { diff --git a/zio/csvio/reader_test.go b/zio/csvio/reader_test.go index 34e927aa30..c474a01245 100644 --- a/zio/csvio/reader_test.go +++ b/zio/csvio/reader_test.go @@ -1,7 +1,6 @@ package csvio import ( - "runtime" "strings" "testing" @@ -11,9 +10,7 @@ import ( func TestNewReaderUsesContextParameter(t *testing.T) { zctx := zed.NewContext() - r := NewReader(zctx, strings.NewReader("f\n1\n"), ReaderOpts{}) - defer runtime.KeepAlive(r) - rec, err := r.Read() + rec, err := NewReader(zctx, strings.NewReader("f\n1\n"), ReaderOpts{}).Read() require.NoError(t, err) typ, err := zctx.LookupType(rec.Type().ID()) require.NoError(t, err) diff --git a/zio/csvio/writer.go b/zio/csvio/writer.go index 02fe32a46c..07b51f7b4a 100644 --- a/zio/csvio/writer.go +++ b/zio/csvio/writer.go @@ -10,6 +10,7 @@ import ( "github.com/brimdata/zed" "github.com/brimdata/zed/runtime/sam/expr" + "github.com/brimdata/zed/zcode" "github.com/brimdata/zed/zson" ) @@ -22,8 +23,6 @@ type Writer struct { types map[int]struct{} first *zed.TypeRecord strings []string - arena *zed.Arena - mapper *zed.Mapper } type WriterOpts struct { @@ -35,14 +34,11 @@ func NewWriter(w io.WriteCloser, opts WriterOpts) *Writer { if opts.Delim != 0 { encoder.Comma = opts.Delim } - zctx := zed.NewContext() return &Writer{ writer: w, encoder: encoder, - flattener: expr.NewFlattener(zctx), + flattener: expr.NewFlattener(zed.NewContext()), types: make(map[int]struct{}), - arena: zed.NewArena(), - mapper: zed.NewMapper(zctx), } } @@ -60,8 +56,7 @@ func (w *Writer) Write(rec zed.Value) error { if rec.Type().Kind() != zed.RecordKind { return fmt.Errorf("CSV output encountered non-record value: %s", zson.FormatValue(rec)) } - w.arena.Reset() - rec, err := w.flattener.Flatten(w.arena, rec) + rec, err := w.flattener.Flatten(rec) if err != nil { return err } @@ -85,24 +80,17 @@ func (w *Writer) Write(rec zed.Value) error { for i, it := 0, rec.Bytes().Iter(); i < len(fields) && !it.Done(); i++ { var s string if zb := it.Next(); zb != nil { - typ, err := w.mapper.Enter(fields[i].Type) - if err != nil { - return err - } - val := w.arena.New(typ, zb).Under(w.arena) + val := zed.NewValue(fields[i].Type, zb).Under() switch id := val.Type().ID(); { case id == zed.IDBytes && len(val.Bytes()) == 0: // We want "" instead of "0x" for a zero-length value. case id == zed.IDString: s = string(val.Bytes()) - case id < zed.IDTypeComplex: - // Avoid ZSON decoration - s = zson.FormatPrimitive(val.Type(), val.Bytes()) + default: + s = formatValue(val.Type(), val.Bytes()) if zed.IsFloat(id) && strings.HasSuffix(s, ".") { s = strings.TrimSuffix(s, ".") } - default: - s = zson.FormatValue(val) } } w.strings = append(w.strings, s) @@ -110,6 +98,14 @@ func (w *Writer) Write(rec zed.Value) error { return w.encoder.Write(w.strings) } +func formatValue(typ zed.Type, bytes zcode.Bytes) string { + // Avoid ZSON decoration. + if typ.ID() < zed.IDTypeComplex { + return zson.FormatPrimitive(zed.TypeUnder(typ), bytes) + } + return zson.FormatValue(zed.NewValue(typ, bytes)) +} + func fieldNamesEqual(a, b []zed.Field) bool { return slices.EqualFunc(a, b, func(a, b zed.Field) bool { return a.Name == b.Name diff --git a/zio/emitter/split.go b/zio/emitter/split.go index ff8a1ecdcb..ec9dd91c08 100644 --- a/zio/emitter/split.go +++ b/zio/emitter/split.go @@ -21,7 +21,6 @@ type Split struct { writers map[zed.Type]zio.WriteCloser seen map[string]struct{} engine storage.Engine - arena *zed.Arena } var _ zio.Writer = (*Split)(nil) @@ -44,7 +43,6 @@ func NewSplit(ctx context.Context, engine storage.Engine, dir *storage.URI, pref writers: make(map[zed.Type]zio.WriteCloser), seen: make(map[string]struct{}), engine: engine, - arena: zed.NewArena(), }, nil } @@ -76,8 +74,7 @@ func (s *Split) lookupOutput(val zed.Value) (zio.WriteCloser, error) { // different Zed types, then we prepend it to the unique ID. func (s *Split) path(r zed.Value) *storage.URI { uniq := strconv.Itoa(len(s.writers)) - s.arena.Reset() - if _path := r.Deref(s.arena, "_path").AsString(); _path != "" { + if _path := r.Deref("_path").AsString(); _path != "" { if _, ok := s.seen[_path]; ok { uniq = _path + "-" + uniq } else { diff --git a/zio/emitter/split_test.go b/zio/emitter/split_test.go index d3f4077138..c9caa58de2 100644 --- a/zio/emitter/split_test.go +++ b/zio/emitter/split_test.go @@ -34,6 +34,7 @@ func TestDirS3Source(t *testing.T) { Return(zio.NopCloser(bytes.NewBuffer(nil)), nil) r := zsonio.NewReader(zed.NewContext(), strings.NewReader(input)) + require.NoError(t, err) w, err := NewSplit(context.Background(), engine, uri, "", false, anyio.WriterOpts{Format: "zson"}) require.NoError(t, err) require.NoError(t, zio.Copy(w, r)) diff --git a/zio/jsonio/builder.go b/zio/jsonio/builder.go index 53201b1b21..eeca026c4f 100644 --- a/zio/jsonio/builder.go +++ b/zio/jsonio/builder.go @@ -10,8 +10,7 @@ import ( ) type builder struct { - arena *zed.Arena - zctx *zed.Context + zctx *zed.Context containers []int // Stack of open containers (as indexes into items). items []item // Stack of items. @@ -30,7 +29,6 @@ type item struct { } func (b *builder) reset() { - b.arena.Reset() b.containers = b.containers[:0] b.items = b.items[:0] } @@ -175,6 +173,6 @@ func (b *builder) value() *zed.Value { panic("multiple items") } item := &b.items[0] - b.val = b.arena.New(item.typ, item.zb.Bytes().Body()) + b.val = zed.NewValue(item.typ, item.zb.Bytes().Body()) return &b.val } diff --git a/zio/jsonio/reader.go b/zio/jsonio/reader.go index e72e902a33..de357e0554 100644 --- a/zio/jsonio/reader.go +++ b/zio/jsonio/reader.go @@ -20,7 +20,7 @@ type Reader struct { func NewReader(zctx *zed.Context, r io.Reader) *Reader { return &Reader{ - builder: builder{arena: zed.NewArena(), zctx: zctx}, + builder: builder{zctx: zctx}, // 64 KB gave the best performance when this was written. lexer: jsonlexer.New(bufio.NewReaderSize(r, 64*1024)), // Ensure handleToken never passes a nil buf to diff --git a/zio/jsonio/writer.go b/zio/jsonio/writer.go index 27f663b794..82ea4ff0db 100644 --- a/zio/jsonio/writer.go +++ b/zio/jsonio/writer.go @@ -30,7 +30,6 @@ type Writer struct { writer *bufio.Writer tab int - arena *zed.Arena // Use json.Encoder for primitive Values. Have to use // json.Encoder instead of json.Marshal because it's // the only way to turn off HTML escaping. @@ -47,7 +46,6 @@ func NewWriter(writer io.WriteCloser, opts WriterOpts) *Writer { Closer: writer, writer: bufio.NewWriter(writer), tab: opts.Pretty, - arena: zed.NewArena(), } w.primEnc = json.NewEncoder(&w.primBuf) w.primEnc.SetEscapeHTML(false) @@ -55,7 +53,6 @@ func NewWriter(writer io.WriteCloser, opts WriterOpts) *Writer { } func (w *Writer) Write(val zed.Value) error { - w.arena.Reset() // writeAny doesn't return an error because any error that occurs will be // surfaced with w.writer.Flush is called. w.writeAny(0, val) @@ -64,7 +61,7 @@ func (w *Writer) Write(val zed.Value) error { } func (w *Writer) writeAny(tab int, val zed.Value) { - val = val.Under(w.arena) + val = val.Under() if val.IsNull() { w.writeColor([]byte("null"), nullColor) return @@ -103,7 +100,7 @@ func (w *Writer) writeRecord(tab int, typ *zed.TypeRecord, bytes zcode.Bytes) { if i != 0 { w.punc(',') } - w.writeEntry(tab, f.Name, w.arena.New(f.Type, it.Next())) + w.writeEntry(tab, f.Name, zed.NewValue(f.Type, it.Next())) } w.newline() w.indent(tab - w.tab) @@ -124,7 +121,7 @@ func (w *Writer) writeArray(tab int, typ zed.Type, bytes zcode.Bytes) { } w.newline() w.indent(tab) - w.writeAny(tab, w.arena.New(typ, it.Next())) + w.writeAny(tab, zed.NewValue(typ, it.Next())) } w.newline() w.indent(tab - w.tab) @@ -144,7 +141,7 @@ func (w *Writer) writeMap(tab int, typ *zed.TypeMap, bytes zcode.Bytes) { w.punc(',') } key := mapKey(typ.KeyType, it.Next()) - w.writeEntry(tab, key, w.arena.New(typ.ValType, it.Next())) + w.writeEntry(tab, key, zed.NewValue(typ.ValType, it.Next())) } w.newline() w.indent(tab - w.tab) @@ -152,22 +149,23 @@ func (w *Writer) writeMap(tab int, typ *zed.TypeMap, bytes zcode.Bytes) { } func mapKey(typ zed.Type, b zcode.Bytes) string { - switch typ.Kind() { + val := zed.NewValue(typ, b) + switch val.Type().Kind() { case zed.PrimitiveKind: - if typ.ID() == zed.IDString { + if val.Type().ID() == zed.IDString { // Don't quote strings. - return zed.DecodeString(b) + return val.AsString() } - return zson.FormatPrimitive(typ, b) + return zson.FormatPrimitive(val.Type(), val.Bytes()) case zed.UnionKind: // Untagged, decorated ZSON so // |{0:1,0(uint64):2,0(=t):3,"0":4}| gets unique keys. typ, bytes := typ.(*zed.TypeUnion).Untag(b) - return zson.FormatTypeAndBytes(typ, bytes) + return zson.FormatValue(zed.NewValue(typ, bytes)) case zed.EnumKind: return convertEnum(typ.(*zed.TypeEnum), b) default: - return zson.FormatTypeAndBytes(typ, b) + return zson.FormatValue(val) } } @@ -185,7 +183,7 @@ func convertEnum(typ *zed.TypeEnum, bytes zcode.Bytes) string { func (w *Writer) writeError(tab int, typ *zed.TypeError, bytes zcode.Bytes) { tab += w.tab w.punc('{') - w.writeEntry(tab, "error", w.arena.New(typ.Type, bytes)) + w.writeEntry(tab, "error", zed.NewValue(typ.Type, bytes)) w.newline() w.indent(tab - w.tab) w.punc('}') diff --git a/zio/lakeio/unmarshal.go b/zio/lakeio/unmarshal.go index 9dc079e268..f0017f5a85 100644 --- a/zio/lakeio/unmarshal.go +++ b/zio/lakeio/unmarshal.go @@ -1,7 +1,6 @@ package lakeio import ( - "github.com/brimdata/zed" "github.com/brimdata/zed/lake" "github.com/brimdata/zed/lake/commits" "github.com/brimdata/zed/lake/data" @@ -11,9 +10,10 @@ import ( "github.com/brimdata/zed/zson" ) -func newUnmarshaler(zctx *zed.Context, arena *zed.Arena) *zson.UnmarshalZNGContext { - unmarshaler := zson.NewZNGUnmarshaler() - unmarshaler.SetContext(zctx, arena) +var unmarshaler *zson.UnmarshalZNGContext + +func init() { + unmarshaler = zson.NewZNGUnmarshaler() unmarshaler.Bind( commits.Add{}, commits.Commit{}, @@ -25,5 +25,4 @@ func newUnmarshaler(zctx *zed.Context, arena *zed.Arena) *zson.UnmarshalZNGConte lake.BranchTip{}, data.Object{}, ) - return unmarshaler } diff --git a/zio/lakeio/writer.go b/zio/lakeio/writer.go index 656ef0f5e8..68f8c9aa85 100644 --- a/zio/lakeio/writer.go +++ b/zio/lakeio/writer.go @@ -56,9 +56,8 @@ func NewWriter(w io.WriteCloser, opts WriterOpts) *Writer { } func (w *Writer) Write(rec zed.Value) error { - arena, _ := rec.Arena() var v interface{} - if err := newUnmarshaler(nil, arena).Unmarshal(rec, &v); err != nil { + if err := unmarshaler.Unmarshal(rec, &v); err != nil { return w.WriteZSON(rec) } var b bytes.Buffer diff --git a/zio/lineio/reader.go b/zio/lineio/reader.go index 00c1a02803..3f217f09ea 100644 --- a/zio/lineio/reader.go +++ b/zio/lineio/reader.go @@ -9,24 +9,19 @@ import ( type Reader struct { scanner *bufio.Scanner - arena *zed.Arena val zed.Value } func NewReader(r io.Reader) *Reader { s := bufio.NewScanner(r) s.Buffer(nil, 25*1024*1024) - return &Reader{ - scanner: bufio.NewScanner(r), - arena: zed.NewArena(), - } + return &Reader{scanner: s} } func (r *Reader) Read() (*zed.Value, error) { if !r.scanner.Scan() || r.scanner.Err() != nil { return nil, r.scanner.Err() } - r.arena.Reset() - r.val = r.arena.NewString(r.scanner.Text()) + r.val = zed.NewString(r.scanner.Text()) return &r.val, nil } diff --git a/zio/peeker_test.go b/zio/peeker_test.go index d62daada92..fab3874a5f 100644 --- a/zio/peeker_test.go +++ b/zio/peeker_test.go @@ -2,7 +2,6 @@ package zio import ( "bytes" - "runtime" "strings" "testing" @@ -21,14 +20,11 @@ func TestPeeker(t *testing.T) { ` stream := zsonio.NewReader(zed.NewContext(), strings.NewReader(input)) peeker := NewPeeker(stream) - defer runtime.KeepAlive(peeker) rec1, err := peeker.Peek() if err != nil { t.Error(err) } - arena := zed.NewArena() - defer arena.Unref() - rec1 = rec1.Copy(arena).Ptr() + rec1 = rec1.Copy().Ptr() rec2, err := peeker.Peek() if err != nil { t.Error(err) @@ -40,7 +36,7 @@ func TestPeeker(t *testing.T) { if err != nil { t.Error(err) } - rec3 = rec3.Copy(arena).Ptr() + rec3 = rec3.Copy().Ptr() if !bytes.Equal(rec1.Bytes(), rec3.Bytes()) { t.Error("rec1 != rec3") } @@ -48,7 +44,7 @@ func TestPeeker(t *testing.T) { if err != nil { t.Error(err) } - rec4 = rec4.Copy(arena).Ptr() + rec4 = rec4.Copy().Ptr() if bytes.Equal(rec3.Bytes(), rec4.Bytes()) { t.Error("rec3 == rec4") } diff --git a/zio/tableio/writer.go b/zio/tableio/writer.go index e8c68a2b71..039ab7cbb9 100644 --- a/zio/tableio/writer.go +++ b/zio/tableio/writer.go @@ -20,17 +20,15 @@ type Writer struct { typ *zed.TypeRecord limit int nline int - arena *zed.Arena } func NewWriter(w io.WriteCloser) *Writer { - zctx := zed.NewContext() + table := tabwriter.NewWriter(w, 0, 8, 1, ' ', 0) return &Writer{ writer: w, - flattener: expr.NewFlattener(zctx), - table: tabwriter.NewWriter(w, 0, 8, 1, ' ', 0), + flattener: expr.NewFlattener(zed.NewContext()), + table: table, limit: 1000, - arena: zed.NewArena(), } } @@ -38,8 +36,7 @@ func (w *Writer) Write(r zed.Value) error { if r.Type().Kind() != zed.RecordKind { return fmt.Errorf("table output encountered non-record value: %s", zson.FormatValue(r)) } - w.arena.Reset() - r, err := w.flattener.Flatten(w.arena, r) + r, err := w.flattener.Flatten(r) if err != nil { return err } @@ -61,13 +58,13 @@ func (w *Writer) Write(r zed.Value) error { var out []string for k, f := range r.Fields() { var v string - value := r.DerefByColumn(w.arena, k).MissingAsNull() + value := r.DerefByColumn(k).MissingAsNull() if f.Type == zed.TypeTime { if !value.IsNull() { v = zed.DecodeTime(value.Bytes()).Time().Format(time.RFC3339Nano) } } else { - v = zeekio.FormatValue(w.arena, value) + v = zeekio.FormatValue(value) } out = append(out, v) } diff --git a/zio/textio/writer.go b/zio/textio/writer.go index 4d9ac326d0..c1045a5584 100644 --- a/zio/textio/writer.go +++ b/zio/textio/writer.go @@ -14,15 +14,12 @@ import ( type Writer struct { writer io.WriteCloser flattener *expr.Flattener - arena *zed.Arena } func NewWriter(w io.WriteCloser) *Writer { - zctx := zed.NewContext() return &Writer{ writer: w, - flattener: expr.NewFlattener(zctx), - arena: zed.NewArena(), + flattener: expr.NewFlattener(zed.NewContext()), } } @@ -31,23 +28,22 @@ func (w *Writer) Close() error { } func (w *Writer) Write(val zed.Value) error { - w.arena.Reset() if _, ok := zed.TypeUnder(val.Type()).(*zed.TypeRecord); ok { return w.writeRecord(val) } - _, err := fmt.Fprintln(w.writer, zeekio.FormatValue(w.arena, val)) + _, err := fmt.Fprintln(w.writer, zeekio.FormatValue(val)) return err } func (w *Writer) writeRecord(rec zed.Value) error { - rec, err := w.flattener.Flatten(w.arena, rec) + rec, err := w.flattener.Flatten(rec) if err != nil { return err } var out []string for k, f := range zed.TypeRecordOf(rec.Type()).Fields { var s string - value := rec.DerefByColumn(w.arena, k).MissingAsNull() + value := rec.DerefByColumn(k).MissingAsNull() if f.Type == zed.TypeTime { if value.IsNull() { s = "-" @@ -55,7 +51,7 @@ func (w *Writer) writeRecord(rec zed.Value) error { s = zed.DecodeTime(value.Bytes()).Time().Format(time.RFC3339Nano) } } else { - s = zeekio.FormatValue(w.arena, value) + s = zeekio.FormatValue(value) } out = append(out, s) } diff --git a/zio/vngio/reader.go b/zio/vngio/reader.go index 055123dd2f..66bd2782bb 100644 --- a/zio/vngio/reader.go +++ b/zio/vngio/reader.go @@ -15,7 +15,7 @@ func NewReader(zctx *zed.Context, r io.Reader, demandOut demand.Demand) (zio.Rea if !ok { return nil, errors.New("VNG requires a seekable input") } - o, err := vng.NewObject(zctx, ra) + o, err := vng.NewObject(ra) if err != nil { return nil, err } diff --git a/zio/zeekio/builder.go b/zio/zeekio/builder.go index 7fbea15a5f..ed522db762 100644 --- a/zio/zeekio/builder.go +++ b/zio/zeekio/builder.go @@ -21,7 +21,7 @@ type builder struct { val zed.Value } -func (b *builder) build(arena *zed.Arena, typ *zed.TypeRecord, sourceFields []int, path []byte, data []byte) (*zed.Value, error) { +func (b *builder) build(typ *zed.TypeRecord, sourceFields []int, path []byte, data []byte) (*zed.Value, error) { b.Truncate() b.Grow(len(data)) fields := typ.Fields @@ -60,7 +60,7 @@ func (b *builder) build(arena *zed.Arena, typ *zed.TypeRecord, sourceFields []in if len(leftoverFields) != 0 { return nil, errors.New("too many values") } - b.val = arena.New(typ, b.Bytes()) + b.val = zed.NewValue(typ, b.Bytes()) return &b.val, nil } diff --git a/zio/zeekio/format.go b/zio/zeekio/format.go index 3b3fbc0a6a..cc30784487 100644 --- a/zio/zeekio/format.go +++ b/zio/zeekio/format.go @@ -15,12 +15,12 @@ import ( "github.com/brimdata/zed/zson" ) -func formatAny(arena *zed.Arena, val zed.Value, inContainer bool) string { +func formatAny(val zed.Value, inContainer bool) string { switch t := val.Type().(type) { case *zed.TypeArray: - return formatArray(arena, t, val.Bytes()) + return formatArray(t, val.Bytes()) case *zed.TypeNamed: - return formatAny(arena, arena.New(t.Type, val.Bytes()), inContainer) + return formatAny(zed.NewValue(t.Type, val.Bytes()), inContainer) case *zed.TypeOfBool: if val.Bool() { return "T" @@ -31,7 +31,7 @@ func formatAny(arena *zed.Arena, val zed.Value, inContainer bool) string { case *zed.TypeOfDuration, *zed.TypeOfTime: return formatTime(nano.Ts(val.Int())) case *zed.TypeEnum: - return formatAny(arena, arena.New(zed.TypeUint64, val.Bytes()), false) + return formatAny(zed.NewValue(zed.TypeUint64, val.Bytes()), false) case *zed.TypeOfFloat16, *zed.TypeOfFloat32: return strconv.FormatFloat(val.Float(), 'f', -1, 32) case *zed.TypeOfFloat64: @@ -43,21 +43,21 @@ func formatAny(arena *zed.Arena, val zed.Value, inContainer bool) string { case *zed.TypeOfIP: return zed.DecodeIP(val.Bytes()).String() case *zed.TypeMap: - return formatMap(arena, t, val.Bytes()) + return formatMap(t, val.Bytes()) case *zed.TypeOfNet: return zed.DecodeNet(val.Bytes()).String() case *zed.TypeOfNull: return "-" case *zed.TypeRecord: - return formatRecord(arena, t, val.Bytes()) + return formatRecord(t, val.Bytes()) case *zed.TypeSet: - return formatSet(arena, t, val.Bytes()) + return formatSet(t, val.Bytes()) case *zed.TypeOfString: return formatString(t, val.Bytes(), inContainer) case *zed.TypeOfType: return zson.String(val) case *zed.TypeUnion: - return formatUnion(arena, t, val.Bytes()) + return formatUnion(t, val.Bytes()) case *zed.TypeError: if zed.TypeUnder(t.Type) == zed.TypeString { return string(val.Bytes()) @@ -68,7 +68,7 @@ func formatAny(arena *zed.Arena, val zed.Value, inContainer bool) string { } } -func formatArray(arena *zed.Arena, t *zed.TypeArray, zv zcode.Bytes) string { +func formatArray(t *zed.TypeArray, zv zcode.Bytes) string { if len(zv) == 0 { return "(empty)" } @@ -87,25 +87,25 @@ func formatArray(arena *zed.Arena, t *zed.TypeArray, zv zcode.Bytes) string { if val := it.Next(); val == nil { b.WriteByte('-') } else { - b.WriteString(formatAny(arena, arena.New(t.Type, val), true)) + b.WriteString(formatAny(zed.NewValue(t.Type, val), true)) } } return b.String() } -func formatMap(arena *zed.Arena, t *zed.TypeMap, zv zcode.Bytes) string { +func formatMap(t *zed.TypeMap, zv zcode.Bytes) string { var b strings.Builder it := zv.Iter() b.WriteByte('[') for !it.Done() { - b.WriteString(formatAny(arena, arena.New(t.KeyType, it.Next()), true)) - b.WriteString(formatAny(arena, arena.New(t.ValType, it.Next()), true)) + b.WriteString(formatAny(zed.NewValue(t.KeyType, it.Next()), true)) + b.WriteString(formatAny(zed.NewValue(t.ValType, it.Next()), true)) } b.WriteByte(']') return b.String() } -func formatRecord(arena *zed.Arena, t *zed.TypeRecord, zv zcode.Bytes) string { +func formatRecord(t *zed.TypeRecord, zv zcode.Bytes) string { var b strings.Builder separator := byte(',') first := true @@ -119,13 +119,13 @@ func formatRecord(arena *zed.Arena, t *zed.TypeRecord, zv zcode.Bytes) string { if val := it.Next(); val == nil { b.WriteByte('-') } else { - b.WriteString(formatAny(arena, arena.New(f.Type, val), false)) + b.WriteString(formatAny(zed.NewValue(f.Type, val), false)) } } return b.String() } -func formatSet(arena *zed.Arena, t *zed.TypeSet, zv zcode.Bytes) string { +func formatSet(t *zed.TypeSet, zv zcode.Bytes) string { if len(zv) == 0 { return "(empty)" } @@ -139,7 +139,7 @@ func formatSet(arena *zed.Arena, t *zed.TypeSet, zv zcode.Bytes) string { } else { b.WriteByte(separator) } - b.WriteString(formatAny(arena, arena.New(t.Type, it.Next()), true)) + b.WriteString(formatAny(zed.NewValue(t.Type, it.Next()), true)) } return b.String() } @@ -189,18 +189,18 @@ func unescape(r rune) []byte { return b.Bytes() } -func formatUnion(arena *zed.Arena, t *zed.TypeUnion, zv zcode.Bytes) string { +func formatUnion(t *zed.TypeUnion, zv zcode.Bytes) string { if zv == nil { - return FormatValue(arena, zed.Null) + return FormatValue(zed.Null) } typ, iv := t.Untag(zv) s := strconv.FormatInt(int64(t.TagOf(typ)), 10) + ":" - return s + formatAny(arena, arena.New(typ, iv), false) + return s + formatAny(zed.NewValue(typ, iv), false) } -func FormatValue(arena *zed.Arena, val zed.Value) string { - if val.IsNull() { +func FormatValue(v zed.Value) string { + if v.IsNull() { return "-" } - return formatAny(arena, val, false) + return formatAny(v, false) } diff --git a/zio/zeekio/parser.go b/zio/zeekio/parser.go index 36a453533a..ad7112cb0c 100644 --- a/zio/zeekio/parser.go +++ b/zio/zeekio/parser.go @@ -363,7 +363,7 @@ func (p *Parser) Descriptor() (*zed.TypeRecord, bool) { } -func (p *Parser) ParseValue(arena *zed.Arena, line []byte) (*zed.Value, error) { +func (p *Parser) ParseValue(line []byte) (*zed.Value, error) { if p.descriptor == nil { if err := p.setDescriptor(); err != nil { return nil, err @@ -375,5 +375,5 @@ func (p *Parser) ParseValue(arena *zed.Arena, line []byte) (*zed.Value, error) { // each time here path = []byte(p.Path) } - return p.builder.build(arena, p.descriptor, p.sourceFields, path, line) + return p.builder.build(p.descriptor, p.sourceFields, path, line) } diff --git a/zio/zeekio/parser_test.go b/zio/zeekio/parser_test.go index 8591ab224d..b56b40c60f 100644 --- a/zio/zeekio/parser_test.go +++ b/zio/zeekio/parser_test.go @@ -31,8 +31,8 @@ func makeHeader(name string, rest []string) string { // startTest() creates a new parser and sends all the provided // directives, expecting them to all be parsed successfully. // A parser object ready for further testing is returned. -func startTest(t *testing.T, zctx *zed.Context, headers []string) *Parser { - p := NewParser(zctx) +func startTest(t *testing.T, headers []string) *Parser { + p := NewParser(zed.NewContext()) for _, h := range headers { require.NoError(t, p.ParseDirective([]byte(h))) } @@ -44,7 +44,7 @@ func startTest(t *testing.T, zctx *zed.Context, headers []string) *Parser { // zeek legacy directives. If any of fields, types, path are provided, // corresponding #files, #types, and #path directives are also sent. // A parser object ready for further testing is returned. -func startLegacyTest(t *testing.T, zctx *zed.Context, fields, types []string, path string) *Parser { +func startLegacyTest(t *testing.T, fields, types []string, path string) *Parser { headers := standardHeaders if len(path) > 0 { headers = append(headers, fmt.Sprintf("#path\t%s", path)) @@ -56,13 +56,13 @@ func startLegacyTest(t *testing.T, zctx *zed.Context, fields, types []string, pa headers = append(headers, makeHeader("#types", types)) } - return startTest(t, zctx, headers) + return startTest(t, headers) } // sendLegacyValues() formats the array of values as a legacy zeek log line // and parses it. -func sendLegacyValues(arena *zed.Arena, p *Parser, vals []string) (*zed.Value, error) { - return p.ParseValue(arena, []byte(strings.Join(vals, "\t"))) +func sendLegacyValues(p *Parser, vals []string) (*zed.Value, error) { + return p.ParseValue([]byte(strings.Join(vals, "\t"))) } func assertError(t *testing.T, err error, pattern, what string) { @@ -73,48 +73,41 @@ func assertError(t *testing.T, err error, pattern, what string) { // Test things related to legacy zeek records that the parser should // handle successfully. func TestLegacyZeekValid(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - // Test standard headers but no timestamp in records - parser := startLegacyTest(t, zctx, fields, types, "") - record, err := sendLegacyValues(arena, parser, values) + parser := startLegacyTest(t, fields, types, "") + record, err := sendLegacyValues(parser, values) require.NoError(t, err) - assert.Equal(t, record.Deref(arena, "ts").MissingAsNull(), zed.Null) + assert.Equal(t, record.Deref("ts").MissingAsNull(), zed.Null) // XXX check contents of other fields? // Test standard headers with a timestamp in records fieldsWithTs := append(fields, "ts") typesWithTs := append(types, "time") - parser = startLegacyTest(t, zctx, fieldsWithTs, typesWithTs, "") + parser = startLegacyTest(t, fieldsWithTs, typesWithTs, "") timestamp := "1573588318384.000" valsWithTs := append(values, timestamp) - record, err = sendLegacyValues(arena, parser, valsWithTs) + record, err = sendLegacyValues(parser, valsWithTs) require.NoError(t, err) expectedTs, err := parseTime([]byte(timestamp)) require.NoError(t, err) - x := record.Deref(arena, "ts").AsTime() + x := record.Deref("ts").AsTime() assert.Equal(t, expectedTs, x, "Timestamp is correct") // Test the #path header - parser = startLegacyTest(t, zctx, fieldsWithTs, typesWithTs, "testpath") - record, err = sendLegacyValues(arena, parser, valsWithTs) + parser = startLegacyTest(t, fieldsWithTs, typesWithTs, "testpath") + record, err = sendLegacyValues(parser, valsWithTs) require.NoError(t, err) - path := record.Deref(arena, "_path").AsString() + path := record.Deref("_path").AsString() assert.Equal(t, path, "testpath", "Legacy _path field was set properly") // XXX test overriding separator, setSeparator } func TestNestedRecords(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - // Test the parser handling of nested records. // The schema used here touches several edge cases: // - nested records separated by a regular field @@ -125,8 +118,8 @@ func TestNestedRecords(t *testing.T) { types := []string{"int", "int", "int", "int", "int", "int", "int"} vals := []string{"1", "2", "3", "4", "5", "6", "7"} - parser := startLegacyTest(t, zed.NewContext(), names, types, "") - record, err := sendLegacyValues(arena, parser, vals) + parser := startLegacyTest(t, names, types, "") + record, err := sendLegacyValues(parser, vals) require.NoError(t, err) require.NoError(t, record.Validate()) @@ -158,77 +151,73 @@ func TestNestedRecords(t *testing.T) { assert.Equal(t, "z", nest3Type.Fields[0].Name, "field in nest3 is z") // Now check the actual values - assert.Equal(t, 1, int(record.Deref(arena, "a").AsInt()), "Field a has value 1") + assert.Equal(t, 1, int(record.Deref("a").AsInt()), "Field a has value 1") - e := record.Deref(arena, "nest1") + e := record.Deref("nest1") assert.Equal(t, nest1Type, e.Type(), "Got right type for field nest1") - assert.Equal(t, 2, int(e.Deref(arena, "a").AsInt()), "nest1.a") - assert.Equal(t, 3, int(e.Deref(arena, "b").AsInt()), "nest1.b") + assert.Equal(t, 2, int(e.Deref("a").AsInt()), "nest1.a") + assert.Equal(t, 3, int(e.Deref("b").AsInt()), "nest1.b") - e = e.Deref(arena, "nestnest") + e = e.Deref("nestnest") assert.Equal(t, nestnestType, e.Type(), "Got right type for field nest1.nestnest") - assert.Equal(t, 6, int(e.Deref(arena, "c").AsInt()), "nest1.nestnest.c") + assert.Equal(t, 6, int(e.Deref("c").AsInt()), "nest1.nestnest.c") - assert.Equal(t, 4, int(record.Deref(arena, "b").AsInt()), "Field b has value 4") + assert.Equal(t, 4, int(record.Deref("b").AsInt()), "Field b has value 4") - e = record.Deref(arena, "nest2") + e = record.Deref("nest2") assert.Equal(t, nest2Type, e.Type(), "Got right type for field nest2") - assert.Equal(t, 5, int(e.Deref(arena, "y").AsInt()), "nest2.y") + assert.Equal(t, 5, int(e.Deref("y").AsInt()), "nest2.y") - e = record.Deref(arena, "nest3") + e = record.Deref("nest3") assert.Equal(t, nest3Type, e.Type(), "Got right type for field nest3") - assert.Equal(t, 7, int(e.Deref(arena, "z").AsInt()), "nest3.z") + assert.Equal(t, 7, int(e.Deref("z").AsInt()), "nest3.z") } // Test things related to legacy zeek records that should cause the // parser to generate errors. func TestLegacyZeekInvalid(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - // Test that a non-standard value for empty_field is rejected - parser := startTest(t, zctx, []string{separator, setSeparator}) + parser := startTest(t, []string{separator, setSeparator}) err := parser.ParseDirective([]byte("#empty_field\tboo")) assertError(t, err, "encountered bad header field", "#empty_field header") // Test that a non-standard value for unset_field is rejected - parser = startTest(t, zctx, []string{separator, setSeparator}) + parser = startTest(t, []string{separator, setSeparator}) err = parser.ParseDirective([]byte("#unset_field\tboo")) assertError(t, err, "encountered bad header field", "#unset header") // Test that missing #fields/#values headers is an error - parser = startTest(t, zctx, standardHeaders) - _, err = sendLegacyValues(arena, parser, values) + parser = startTest(t, standardHeaders) + _, err = sendLegacyValues(parser, values) assertError(t, err, "bad types/fields definition", "missing #fields/#types header") // Test that #fields header without #values is an error fh := makeHeader("#fields", fields) - parser = startTest(t, zctx, append(standardHeaders, fh)) - _, err = sendLegacyValues(arena, parser, values) + parser = startTest(t, append(standardHeaders, fh)) + _, err = sendLegacyValues(parser, values) assertError(t, err, "bad types/fields definition", "missing #types header") // Test that #types header without #fields is an error th := makeHeader("#types", types) - parser = startTest(t, zctx, append(standardHeaders, th)) - _, err = sendLegacyValues(arena, parser, values) + parser = startTest(t, append(standardHeaders, th)) + _, err = sendLegacyValues(parser, values) assertError(t, err, "bad types/fields definition", "values without #fields") // Test that mismatched #fields/#types headers is an error /* XXX fixme - parser = startTest(t, zctx, append(standardHeaders, fh)) + parser = startTest(t, append(standardHeaders, fh)) err = parser.parseDirective([]byte(makeHeader("#types", append(types, "int")))) assertError(t, err, "bad types/fields definition", "mismatched #fields/#types headers") */ // Test that too many values is an error - parser = startTest(t, zctx, append(standardHeaders, fh, th)) - _, err = sendLegacyValues(arena, parser, append(values, "extra")) + parser = startTest(t, append(standardHeaders, fh, th)) + _, err = sendLegacyValues(parser, append(values, "extra")) assertError(t, err, "too many values", "wrong number of values") // Test that too few values is an error - parser = startTest(t, zctx, append(standardHeaders, fh, th)) - _, err = sendLegacyValues(arena, parser, values[:len(values)-2]) + parser = startTest(t, append(standardHeaders, fh, th)) + _, err = sendLegacyValues(parser, values[:len(values)-2]) assertError(t, err, "too few values", "wrong number of values") // XXX check invalid types? diff --git a/zio/zeekio/reader.go b/zio/zeekio/reader.go index 41c1a7f5e7..ddf8e53f2f 100644 --- a/zio/zeekio/reader.go +++ b/zio/zeekio/reader.go @@ -15,7 +15,6 @@ const ( ) type Reader struct { - arena *zed.Arena scanner *skim.Scanner parser *Parser } @@ -23,7 +22,6 @@ type Reader struct { func NewReader(zctx *zed.Context, reader io.Reader) *Reader { buffer := make([]byte, ReadSize) return &Reader{ - arena: zed.NewArena(), scanner: skim.NewScanner(reader, buffer, MaxLineSize), parser: NewParser(zctx), } @@ -54,8 +52,7 @@ again: } goto again } - r.arena.Reset() - rec, err := r.parser.ParseValue(r.arena, line) + rec, err := r.parser.ParseValue(line) if err != nil { return nil, e(err) } diff --git a/zio/zeekio/reader_test.go b/zio/zeekio/reader_test.go index 058e99223f..9d42ca17c9 100644 --- a/zio/zeekio/reader_test.go +++ b/zio/zeekio/reader_test.go @@ -1,7 +1,6 @@ package zeekio import ( - "runtime" "strings" "testing" "time" @@ -13,9 +12,6 @@ import ( ) func TestReaderCRLF(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - input := ` #separator \x09 #set_separator , @@ -28,12 +24,11 @@ func TestReaderCRLF(t *testing.T) { ` input = strings.ReplaceAll(input, "\n", "\r\n") r := NewReader(zed.NewContext(), strings.NewReader(input)) - defer runtime.KeepAlive(r) rec, err := r.Read() require.NoError(t, err) - ts := rec.Deref(arena, "ts").AsTime() + ts := rec.Deref("ts").AsTime() assert.Exactly(t, 10*nano.Ts(time.Second), ts) - d := rec.Deref(arena, "i").AsInt() + d := rec.Deref("i").AsInt() assert.Exactly(t, int64(1), d) rec, err = r.Read() require.NoError(t, err) diff --git a/zio/zeekio/writer.go b/zio/zeekio/writer.go index 6173069b07..5cf0ba8ea6 100644 --- a/zio/zeekio/writer.go +++ b/zio/zeekio/writer.go @@ -16,19 +16,12 @@ type Writer struct { header flattener *expr.Flattener typ *zed.TypeRecord - zctx *zed.Context - arena *zed.Arena - mapper *zed.Mapper } func NewWriter(w io.WriteCloser) *Writer { - zctx := zed.NewContext() return &Writer{ writer: w, - flattener: expr.NewFlattener(zctx), - zctx: zctx, - arena: zed.NewArena(), - mapper: zed.NewMapper(zctx), + flattener: expr.NewFlattener(zed.NewContext()), } } @@ -37,12 +30,11 @@ func (w *Writer) Close() error { } func (w *Writer) Write(r zed.Value) error { - w.arena.Reset() - r, err := w.flattener.Flatten(w.arena, r) + r, err := w.flattener.Flatten(r) if err != nil { return err } - path := r.Deref(w.arena, "_path").AsString() + path := r.Deref("_path").AsString() if r.Type() != w.typ || path != w.Path { if err := w.writeHeader(r, path); err != nil { return err @@ -61,11 +53,7 @@ func (w *Writer) Write(r zed.Value) error { w.buf.WriteByte('\t') } needSeparator = true - typ, err := w.mapper.Enter(f.Type) - if err != nil { - return err - } - w.buf.WriteString(FormatValue(w.arena, w.arena.New(typ, bytes))) + w.buf.WriteString(FormatValue(zed.NewValue(f.Type, bytes))) } w.buf.WriteByte('\n') _, err = w.writer.Write(w.buf.Bytes()) diff --git a/zio/zjsonio/reader.go b/zio/zjsonio/reader.go index a0d33c1b8b..45a7a76e4d 100644 --- a/zio/zjsonio/reader.go +++ b/zio/zjsonio/reader.go @@ -19,7 +19,6 @@ const ( type Reader struct { scanner *skim.Scanner - arena *zed.Arena zctx *zed.Context decoder decoder builder *zcode.Builder @@ -30,7 +29,6 @@ func NewReader(zctx *zed.Context, reader io.Reader) *Reader { buffer := make([]byte, ReadSize) return &Reader{ scanner: skim.NewScanner(reader, buffer, MaxLineSize), - arena: zed.NewArena(), zctx: zctx, decoder: make(decoder), builder: zcode.NewBuilder(), @@ -61,8 +59,7 @@ func (r *Reader) Read() (*zed.Value, error) { if err := r.decodeValue(r.builder, typ, object.Value); err != nil { return nil, e(err) } - r.arena.Reset() - r.val = r.arena.New(typ, r.builder.Bytes().Body()) + r.val = zed.NewValue(typ, r.builder.Bytes().Body()) return &r.val, nil } @@ -101,7 +98,8 @@ func (r *Reader) decodeValue(b *zcode.Builder, typ zed.Type, body interface{}) e if err != nil { return err } - b.Append(zed.EncodeTypeValue(local)) + tv := r.zctx.LookupTypeValue(local) + b.Append(tv.Bytes()) return nil default: return r.decodePrimitive(b, typ, body) diff --git a/zio/zngio/batch.go b/zio/zngio/batch.go new file mode 100644 index 0000000000..d15508e4ae --- /dev/null +++ b/zio/zngio/batch.go @@ -0,0 +1,62 @@ +package zngio + +import ( + "slices" + "sync" + "sync/atomic" + + "github.com/brimdata/zed" + "github.com/brimdata/zed/zbuf" +) + +type batch struct { + buf *buffer + refs int32 + vals []zed.Value +} + +var _ zbuf.Batch = (*batch)(nil) + +var batchPool sync.Pool + +func newBatch(buf *buffer) *batch { + b, ok := batchPool.Get().(*batch) + if !ok { + b = &batch{vals: make([]zed.Value, 200)} + } + b.buf = buf + b.refs = 1 + b.vals = b.vals[:0] + return b +} + +func (b *batch) extend() *zed.Value { + n := len(b.vals) + b.vals = slices.Grow(b.vals, 1)[:n+1] + return &b.vals[n] +} + +// unextend undoes what extend did. +func (b *batch) unextend() { + b.vals = b.vals[:len(b.vals)-1] +} + +func (b *batch) Ref() { atomic.AddInt32(&b.refs, 1) } + +func (b *batch) Unref() { + if refs := atomic.AddInt32(&b.refs, -1); refs == 0 { + if b.buf != nil { + b.buf.free() + b.buf = nil + } + batchPool.Put(b) + } else if refs < 0 { + panic("zngio: negative batch reference count") + } +} + +func (b *batch) Values() []zed.Value { return b.vals } + +// XXX this should be ok, but we should handle nil receiver in scope so push +// will do the right thing +func (*batch) Vars() []zed.Value { return nil } diff --git a/zio/zngio/buffer.go b/zio/zngio/buffer.go index 41f471016d..203d5e18c4 100644 --- a/zio/zngio/buffer.go +++ b/zio/zngio/buffer.go @@ -41,8 +41,6 @@ func newBufferFromBytes(b []byte) *buffer { return buf } -func (b *buffer) Free() { b.free() } - func (b *buffer) free() { if b == nil { return diff --git a/zio/zngio/scanner.go b/zio/zngio/scanner.go index ed56fdc601..3c29e65173 100644 --- a/zio/zngio/scanner.go +++ b/zio/zngio/scanner.go @@ -177,7 +177,6 @@ func (s *scanner) Progress() zbuf.Progress { // be safely used without any channel involvement. type worker struct { ctx context.Context - zctx *zed.Context progress *zbuf.Progress workCh chan work bufferFilter *expr.BufferFilter @@ -204,7 +203,7 @@ func newWorker(ctx context.Context, p *zbuf.Progress, bf *expr.BufferFilter, f e workCh: make(chan work), bufferFilter: bf, filter: f, - ectx: expr.NewContext(zed.NewArena()), + ectx: expr.NewContext(), validate: validate, } } @@ -252,8 +251,6 @@ func (w *worker) run(ctx context.Context, workerCh chan<- *worker) { } } -var valsPool = sync.Pool{New: func() any { return make([]zed.Value, 256) }} - func (w *worker) scanBatch(buf *buffer, local localctx) (zbuf.Batch, error) { // If w.bufferFilter evaluates to false, we know buf cannot contain // records matching w.filter. @@ -270,74 +267,85 @@ func (w *worker) scanBatch(buf *buffer, local localctx) (zbuf.Batch, error) { // pools of buffers based on size? w.mapperLookupCache.Reset(local.mapper) - arena := zed.NewArenaWithBuffer(buf.Bytes(), buf.Free) - defer arena.Unref() - vals := valsPool.Get().([]zed.Value)[:0] + batch := newBatch(buf) var progress zbuf.Progress - w.ectx.Arena().Reset() + // We extend the batch one past its end and decode into the next + // potential slot and only advance the batch when we decide we want to + // keep the value. Since we overshoot by one slot on every pass, + // we delete the overshoot with batch.unextend() on exit from the loop. + // I think this is what I drew on the Lawton basement whiteboard + // in 2018 but my previous attempts implementing that picture were + // horrible. This attempts isn't so bad. + valRef := batch.extend() for buf.length() > 0 { - val, length, err := w.decodeVal(arena, buf) - if err != nil { - return nil, errBadFormat + if err := w.decodeVal(buf, valRef); err != nil { + buf.free() + return nil, err } - if w.wantValue(val, length, &progress) { - vals = append(vals, val) + if w.wantValue(*valRef, &progress) { + valRef = batch.extend() } } + batch.unextend() w.progress.Add(progress) - if len(vals) == 0 { + if len(batch.Values()) == 0 { + batch.Unref() return nil, nil } - return zbuf.NewBatchWithVarsAndFree(arena, vals, nil, func() { valsPool.Put(vals) }), nil + return batch, nil } -func (w *worker) decodeVal(arena *zed.Arena, buf *buffer) (zed.Value, int64, error) { +func (w *worker) decodeVal(buf *buffer, valRef *zed.Value) error { id, err := readUvarintAsInt(buf) if err != nil { - return zed.Null, 0, err + return err } n, err := zcode.ReadTag(buf) if err != nil { - return zed.Null, 0, errBadFormat - } - typ := w.mapperLookupCache.Lookup(id) - if typ == nil { - return zed.Null, 0, fmt.Errorf("zngio: type ID %d not in context", id) - } - if n < 0 { - return arena.New(typ, nil), 0, nil + return errBadFormat } - off := buf.off - if n > 0 { - if _, err := buf.read(n); err != nil && err != io.EOF { + var b []byte + if n == 0 { + b = []byte{} + } else if n > 0 { + b, err = buf.read(n) + if err != nil && err != io.EOF { if err == peeker.ErrBufferOverflow { - return zed.Null, 0, fmt.Errorf("large value of %d bytes exceeds maximum read buffer", n) + return fmt.Errorf("large value of %d bytes exceeds maximum read buffer", n) } - return zed.Null, 0, errBadFormat + return errBadFormat } } - val := arena.NewFromOffsetAndLength(typ, off, n) + typ := w.mapperLookupCache.Lookup(id) + if typ == nil { + return fmt.Errorf("zngio: type ID %d not in context", id) + } + *valRef = zed.NewValue(typ, b) if w.validate { - if err := val.Validate(); err != nil { - return zed.Null, 0, err + if err := valRef.Validate(); err != nil { + return err } } - return val, int64(n), nil + return nil } -func (w *worker) wantValue(val zed.Value, length int64, progress *zbuf.Progress) bool { - progress.BytesRead += length +func (w *worker) wantValue(val zed.Value, progress *zbuf.Progress) bool { + progress.BytesRead += int64(len(val.Bytes())) progress.RecordsRead++ - if f := w.filter; f != nil { - // It's tempting to call w.bufferFilter.Eval on val.Bytes here, - // but that might call FieldNameFinder.Find, which could explode - // or return false negatives because it expects a sequence of - // (type ID, tag, ZNG value) but val.Bytes is just a ZNG value. - if !f.Eval(w.ectx, val).Ptr().AsBool() { - return false - } + // It's tempting to call w.bufferFilter.Eval on rec.Bytes here, but that + // might call FieldNameFinder.Find, which could explode or return false + // negatives because it expects a buffer of ZNG value messages, and + // rec.Bytes is just a ZNG value. (A ZNG value message is a header + // indicating a type ID followed by a value of that type.) + if w.filter == nil || check(w.ectx, val, w.filter) { + progress.BytesMatched += int64(len(val.Bytes())) + progress.RecordsMatched++ + return true } - progress.BytesMatched += length - progress.RecordsMatched++ - return true + return false +} + +func check(ectx expr.Context, this zed.Value, filter expr.Evaluator) bool { + val := filter.Eval(ectx, this) + return val.Type() == zed.TypeBool && val.Bool() } diff --git a/zio/zngio/scanner_test.go b/zio/zngio/scanner_test.go index 580fc69df2..8dcb7486a9 100644 --- a/zio/zngio/scanner_test.go +++ b/zio/zngio/scanner_test.go @@ -15,10 +15,6 @@ import ( ) func TestScannerContext(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - m := zson.NewZNGMarshaler() - // We want to maximize the number of scanner goroutines running // concurrently, so don't call t.Parallel. count := runtime.GOMAXPROCS(0) + 1 @@ -31,7 +27,7 @@ func TestScannerContext(t *testing.T) { for i := 0; i < count; i++ { names = append(names, strconv.Itoa(i)) values = append(values, i) - rec, err := m.MarshalCustom(arena, names, values) + rec, err := zson.NewZNGMarshaler().MarshalCustom(names, values) require.NoError(t, err) var buf bytes.Buffer w := NewWriter(zio.NopCloser(&buf)) diff --git a/zio/zsonio/reader.go b/zio/zsonio/reader.go index f469963a99..0709ea7e38 100644 --- a/zio/zsonio/reader.go +++ b/zio/zsonio/reader.go @@ -10,7 +10,6 @@ import ( type Reader struct { reader io.Reader - arena *zed.Arena zctx *zed.Context parser *zson.Parser analyzer zson.Analyzer @@ -21,7 +20,6 @@ type Reader struct { func NewReader(zctx *zed.Context, r io.Reader) *Reader { return &Reader{ reader: r, - arena: zed.NewArena(), zctx: zctx, analyzer: zson.NewAnalyzer(), builder: zcode.NewBuilder(), @@ -40,7 +38,6 @@ func (r *Reader) Read() (*zed.Value, error) { if err != nil { return nil, err } - r.arena.Reset() - r.val, err = zson.Build(r.arena, r.builder, val) + r.val, err = zson.Build(r.builder, val) return &r.val, err } diff --git a/zio/zsonio/reader_test.go b/zio/zsonio/reader_test.go index 1b59ca6ccd..87b8e6b278 100644 --- a/zio/zsonio/reader_test.go +++ b/zio/zsonio/reader_test.go @@ -13,9 +13,6 @@ import ( ) func TestReadOneLineNoEOF(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - const expected = `{msg:"record1"}` type result struct { err error @@ -29,9 +26,6 @@ func TestReadOneLineNoEOF(t *testing.T) { reader <- []byte(expected + "\n" + expected) r := zsonio.NewReader(zed.NewContext(), reader) rec, err := r.Read() - if rec != nil { - rec = rec.Copy(arena).Ptr() - } done <- result{val: rec, err: err} }() select { diff --git a/zngbytes/deserializer.go b/zngbytes/deserializer.go index 21043fbb82..52bd1e1a8f 100644 --- a/zngbytes/deserializer.go +++ b/zngbytes/deserializer.go @@ -13,9 +13,12 @@ type Deserializer struct { unmarshaler *zson.UnmarshalZNGContext } -func NewDeserializer(zctx *zed.Context, arena *zed.Arena, reader io.Reader, templates []interface{}) *Deserializer { +func NewDeserializer(reader io.Reader, templates []interface{}) *Deserializer { + return NewDeserializerWithContext(zed.NewContext(), reader, templates) +} + +func NewDeserializerWithContext(zctx *zed.Context, reader io.Reader, templates []interface{}) *Deserializer { u := zson.NewZNGUnmarshaler() - u.SetContext(zctx, arena) u.Bind(templates...) return &Deserializer{ reader: zngio.NewReader(zctx, reader), diff --git a/zngbytes/serializer.go b/zngbytes/serializer.go index 29ee601aed..88919f8301 100644 --- a/zngbytes/serializer.go +++ b/zngbytes/serializer.go @@ -3,7 +3,6 @@ package zngbytes import ( "bytes" - "github.com/brimdata/zed" "github.com/brimdata/zed/zio" "github.com/brimdata/zed/zio/zngio" "github.com/brimdata/zed/zson" @@ -11,18 +10,15 @@ import ( type Serializer struct { marshaler *zson.MarshalZNGContext - arena *zed.Arena buffer bytes.Buffer writer *zngio.Writer } func NewSerializer() *Serializer { - zctx := zed.NewContext() - m := zson.NewZNGMarshalerWithContext(zctx) + m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) s := &Serializer{ marshaler: m, - arena: zed.NewArena(), } s.writer = zngio.NewWriter(zio.NopCloser(&s.buffer)) return s @@ -33,8 +29,7 @@ func (s *Serializer) Decorate(style zson.TypeStyle) { } func (s *Serializer) Write(v interface{}) error { - s.arena.Reset() - rec, err := s.marshaler.Marshal(s.arena, v) + rec, err := s.marshaler.Marshal(v) if err != nil { return err } diff --git a/zson/builder.go b/zson/builder.go index 345224d91a..fa536f18cc 100644 --- a/zson/builder.go +++ b/zson/builder.go @@ -15,13 +15,13 @@ import ( "golang.org/x/text/unicode/norm" ) -func Build(arena *zed.Arena, b *zcode.Builder, val Value) (zed.Value, error) { +func Build(b *zcode.Builder, val Value) (zed.Value, error) { b.Truncate() if err := buildValue(b, val); err != nil { return zed.Null, err } it := b.Bytes().Iter() - return arena.New(val.TypeOf(), it.Next()), nil + return zed.NewValue(val.TypeOf(), it.Next()), nil } func buildValue(b *zcode.Builder, val Value) error { diff --git a/zson/formatter.go b/zson/formatter.go index df2cc7d9c3..aa46e63a15 100644 --- a/zson/formatter.go +++ b/zson/formatter.go @@ -170,9 +170,9 @@ func (f *Formatter) formatValue(indent int, typ zed.Type, bytes zcode.Bytes, par case *zed.TypeRecord: f.formatRecord(indent, t, bytes, known, parentImplied) case *zed.TypeArray: - null = f.formatVector(indent, "[", "]", t.Type, t, bytes, known, parentImplied) + null = f.formatVector(indent, "[", "]", t.Type, zed.NewValue(t, bytes), known, parentImplied) case *zed.TypeSet: - null = f.formatVector(indent, "|[", "]|", t.Type, t, bytes, known, parentImplied) + null = f.formatVector(indent, "|[", "]|", t.Type, zed.NewValue(t, bytes), known, parentImplied) case *zed.TypeUnion: f.formatUnion(indent, t, bytes) case *zed.TypeMap: @@ -420,16 +420,21 @@ func (f *Formatter) formatRecord(indent int, typ *zed.TypeRecord, bytes zcode.By f.indent(indent-f.tab, "}") } -func (f *Formatter) formatVector(indent int, open, close string, inner, typ zed.Type, bytes zcode.Bytes, known, parentImplied bool) bool { +func (f *Formatter) formatVector(indent int, open, close string, inner zed.Type, val zed.Value, known, parentImplied bool) bool { f.build(open) - if len(bytes) == 0 { + n, err := val.ContainerLength() + if err != nil { + panic(err) + } + if n == 0 { f.build(close) return true } indent += f.tab sep := f.newline + it := val.Iter() elems := newElemBuilder(inner) - for it := bytes.Iter(); !it.Done(); { + for !it.Done() { f.build(sep) f.indent(indent, "") typ, b := elems.add(it.Next()) @@ -441,7 +446,7 @@ func (f *Formatter) formatVector(indent int, open, close string, inner, typ zed. if elems.needsDecoration() { // If we haven't seen all the types in the union, print the decorator // so the fullness of the union is persevered. - f.decorate(typ, false, true) + f.decorate(val.Type(), false, true) } return false } @@ -831,9 +836,3 @@ func FormatTypeValue(tv zcode.Bytes) string { f.formatTypeValue(0, tv) return f.builder.String() } - -func FormatTypeAndBytes(typ zed.Type, bytes zcode.Bytes) string { - f := NewFormatter(0, true, nil) - f.formatValueAndDecorate(typ, bytes) - return f.builder.String() -} diff --git a/zson/marshal.go b/zson/marshal.go index b370dcc87e..2f411fd30f 100644 --- a/zson/marshal.go +++ b/zson/marshal.go @@ -24,7 +24,6 @@ func Marshal(v interface{}) (string, error) { type MarshalContext struct { *MarshalZNGContext - arena *zed.Arena formatter *Formatter } @@ -35,7 +34,6 @@ func NewMarshaler() *MarshalContext { func NewMarshalerIndent(indent int) *MarshalContext { return &MarshalContext{ MarshalZNGContext: NewZNGMarshaler(), - arena: zed.NewArena(), formatter: NewFormatter(indent, false, nil), } } @@ -43,13 +41,11 @@ func NewMarshalerIndent(indent int) *MarshalContext { func NewMarshalerWithContext(zctx *zed.Context) *MarshalContext { return &MarshalContext{ MarshalZNGContext: NewZNGMarshalerWithContext(zctx), - arena: zed.NewArena(), } } func (m *MarshalContext) Marshal(v interface{}) (string, error) { - m.arena.Reset() - val, err := m.MarshalZNGContext.Marshal(m.arena, v) + val, err := m.MarshalZNGContext.Marshal(v) if err != nil { return "", err } @@ -57,8 +53,7 @@ func (m *MarshalContext) Marshal(v interface{}) (string, error) { } func (m *MarshalContext) MarshalCustom(names []string, fields []interface{}) (string, error) { - m.arena.Reset() - rec, err := m.MarshalZNGContext.MarshalCustom(m.arena, names, fields) + rec, err := m.MarshalZNGContext.MarshalCustom(names, fields) if err != nil { return "", err } @@ -68,7 +63,6 @@ func (m *MarshalContext) MarshalCustom(names []string, fields []interface{}) (st type UnmarshalContext struct { *UnmarshalZNGContext zctx *zed.Context - arena *zed.Arena analyzer Analyzer builder *zcode.Builder } @@ -77,7 +71,6 @@ func NewUnmarshaler() *UnmarshalContext { return &UnmarshalContext{ UnmarshalZNGContext: NewZNGUnmarshaler(), zctx: zed.NewContext(), - arena: zed.NewArena(), analyzer: NewAnalyzer(), builder: zcode.NewBuilder(), } @@ -97,8 +90,7 @@ func (u *UnmarshalContext) Unmarshal(zson string, v interface{}) error { if err != nil { return err } - u.arena.Reset() - zedVal, err := Build(u.arena, u.builder, val) + zedVal, err := Build(u.builder, val) if err != nil { return nil } @@ -109,8 +101,8 @@ type ZNGMarshaler interface { MarshalZNG(*MarshalZNGContext) (zed.Type, error) } -func MarshalZNG(zctx *zed.Context, arena *zed.Arena, v interface{}) (zed.Value, error) { - return NewZNGMarshalerWithContext(zctx).Marshal(arena, v) +func MarshalZNG(v interface{}) (zed.Value, error) { + return NewZNGMarshaler().Marshal(v) } type MarshalZNGContext struct { @@ -136,7 +128,7 @@ func (m *MarshalZNGContext) MarshalValue(v interface{}) (zed.Type, error) { return m.encodeValue(reflect.ValueOf(v)) } -func (m *MarshalZNGContext) Marshal(arena *zed.Arena, v interface{}) (zed.Value, error) { +func (m *MarshalZNGContext) Marshal(v interface{}) (zed.Value, error) { m.Builder.Reset() typ, err := m.encodeValue(reflect.ValueOf(v)) if err != nil { @@ -147,10 +139,10 @@ func (m *MarshalZNGContext) Marshal(arena *zed.Arena, v interface{}) (zed.Value, if it.Done() { return zed.Null, errors.New("no value found") } - return arena.New(typ, it.Next()), nil + return zed.NewValue(typ, it.Next()), nil } -func (m *MarshalZNGContext) MarshalCustom(arena *zed.Arena, names []string, vals []interface{}) (zed.Value, error) { +func (m *MarshalZNGContext) MarshalCustom(names []string, vals []interface{}) (zed.Value, error) { if len(names) != len(vals) { return zed.Null, errors.New("names and vals have different lengths") } @@ -173,7 +165,7 @@ func (m *MarshalZNGContext) MarshalCustom(arena *zed.Arena, names []string, vals if err != nil { return zed.Null, err } - return arena.New(recType, m.Builder.Bytes()), nil + return zed.NewValue(recType, m.Builder.Bytes()), nil } const ( @@ -302,8 +294,9 @@ func (m *MarshalZNGContext) encodeAny(v reflect.Value) (zed.Type, error) { m.Builder.Append(zed.EncodeTime(nano.TimeToTs(v))) return zed.TypeTime, nil case zed.Type: - m.Builder.Append(zed.EncodeTypeValue(v)) - return zed.TypeType, nil + val := m.Context.LookupTypeValue(v) + m.Builder.Append(val.Bytes()) + return val.Type(), nil case zed.Value: typ, err := m.TranslateType(v.Type()) if err != nil { @@ -644,7 +637,6 @@ type ZNGUnmarshaler interface { type UnmarshalZNGContext struct { zctx *zed.Context - arena *zed.Arena binder binder } @@ -652,10 +644,8 @@ func NewZNGUnmarshaler() *UnmarshalZNGContext { return &UnmarshalZNGContext{} } -func UnmarshalZNG(zctx *zed.Context, arena *zed.Arena, val zed.Value, v interface{}) error { - u := NewZNGUnmarshaler() - u.SetContext(zctx, arena) - return u.Unmarshal(val, v) +func UnmarshalZNG(val zed.Value, v interface{}) error { + return NewZNGUnmarshaler().decodeAny(val, reflect.ValueOf(v)) } func incompatTypeError(zt zed.Type, v reflect.Value) error { @@ -664,10 +654,8 @@ func incompatTypeError(zt zed.Type, v reflect.Value) error { // SetContext provides an optional type context to the unmarshaler. This is // needed only when unmarshaling Zed type values into Go zed.Type interface values. -func (u *UnmarshalZNGContext) SetContext(zctx *zed.Context, arena *zed.Arena) *UnmarshalZNGContext { +func (u *UnmarshalZNGContext) SetContext(zctx *zed.Context) { u.zctx = zctx - u.arena = arena - return u } func (u *UnmarshalZNGContext) Unmarshal(val zed.Value, v interface{}) error { @@ -703,7 +691,7 @@ func (u *UnmarshalZNGContext) NamedBindings(bindings []Binding) error { var netipAddrType = reflect.TypeOf(netip.Addr{}) var netIPType = reflect.TypeOf(net.IP{}) -func (u *UnmarshalZNGContext) decodeAny(val zed.Value, v reflect.Value) error { +func (u *UnmarshalZNGContext) decodeAny(val zed.Value, v reflect.Value) (x error) { if !v.IsValid() { return errors.New("cannot unmarshal into value provided") } @@ -725,12 +713,9 @@ func (u *UnmarshalZNGContext) decodeAny(val zed.Value, v reflect.Value) error { v.Set(reflect.ValueOf(zed.DecodeTime(val.Bytes()))) return nil case zed.Value: - if u.arena == nil { - return errors.New("cannot unmarshal zed.Value without arena") - } // For zed.Values we simply set the reflect value to the // zed.Value that has been decoded. - v.Set(reflect.ValueOf(val.Copy(u.arena))) + v.Set(reflect.ValueOf(val.Copy())) return nil } if zed.TypeUnder(val.Type()) == zed.TypeNull { @@ -931,17 +916,15 @@ func (u *UnmarshalZNGContext) decodeMap(val zed.Value, mapVal reflect.Value) err if mapVal.IsNil() { mapVal.Set(reflect.MakeMap(mapVal.Type())) } - arena := zed.NewArena() - defer arena.Unref() keyType := mapVal.Type().Key() valType := mapVal.Type().Elem() for it := val.Iter(); !it.Done(); { key := reflect.New(keyType).Elem() - if err := u.decodeAny(arena.New(typ.KeyType, it.Next()), key); err != nil { + if err := u.decodeAny(zed.NewValue(typ.KeyType, it.Next()), key); err != nil { return err } val := reflect.New(valType).Elem() - if err := u.decodeAny(arena.New(typ.ValType, it.Next()), val); err != nil { + if err := u.decodeAny(zed.NewValue(typ.ValType, it.Next()), val); err != nil { return err } mapVal.SetMapIndex(key, val) @@ -950,11 +933,9 @@ func (u *UnmarshalZNGContext) decodeMap(val zed.Value, mapVal reflect.Value) err } func (u *UnmarshalZNGContext) decodeRecord(val zed.Value, sval reflect.Value) error { - arena := zed.NewArena() - defer arena.Unref() if union, ok := val.Type().(*zed.TypeUnion); ok { typ, bytes := union.Untag(val.Bytes()) - val = arena.New(typ, bytes) + val = zed.NewValue(typ, bytes) } recType, ok := zed.TypeUnder(val.Type()).(*zed.TypeRecord) if !ok { @@ -975,7 +956,7 @@ func (u *UnmarshalZNGContext) decodeRecord(val zed.Value, sval reflect.Value) er name := recType.Fields[i].Name if fieldIdx, ok := nameToField[name]; ok { typ := recType.Fields[i].Type - if err := u.decodeAny(arena.New(typ, itzv), sval.Field(fieldIdx)); err != nil { + if err := u.decodeAny(zed.NewValue(typ, itzv), sval.Field(fieldIdx)); err != nil { return err } } @@ -1006,8 +987,6 @@ func (u *UnmarshalZNGContext) decodeArray(val zed.Value, arrVal reflect.Value) e arrVal.Set(reflect.Zero(arrVal.Type())) return nil } - arena := zed.NewArena() - defer arena.Unref() i := 0 for it := val.Iter(); !it.Done(); i++ { itzv := it.Next() @@ -1023,7 +1002,7 @@ func (u *UnmarshalZNGContext) decodeArray(val zed.Value, arrVal reflect.Value) e if i >= arrVal.Len() { arrVal.SetLen(i + 1) } - if err := u.decodeAny(arena.New(arrType.Type, itzv), arrVal.Index(i)); err != nil { + if err := u.decodeAny(zed.NewValue(arrType.Type, itzv), arrVal.Index(i)); err != nil { return err } } diff --git a/zson/marshal_test.go b/zson/marshal_test.go index 7aae8a907b..f4a771b144 100644 --- a/zson/marshal_test.go +++ b/zson/marshal_test.go @@ -104,44 +104,42 @@ type SliceRecord struct { } func TestBytes(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - - m := zson.NewZNGMarshalerWithContext(zctx) - rec, err := m.Marshal(arena, BytesRecord{B: []byte{1, 2, 3}}) + m := zson.NewZNGMarshaler() + rec, err := m.Marshal(BytesRecord{B: []byte{1, 2, 3}}) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, "{B:0x010203}", zson.FormatValue(rec)) - rec, err = m.Marshal(arena, BytesArrayRecord{A: [3]byte{4, 5, 6}}) + rec, err = m.Marshal(BytesArrayRecord{A: [3]byte{4, 5, 6}}) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, "{A:0x040506}", zson.FormatValue(rec)) id := IDRecord{A: ID{0, 1, 2, 3}, B: ID{4, 5, 6, 7}} - m = zson.NewZNGMarshalerWithContext(zctx) + m = zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) - rec, err = m.Marshal(arena, id) + rec, err = m.Marshal(id) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, "{A:0x00010203(=ID),B:0x04050607(ID)}(=IDRecord)", zson.FormatValue(rec)) var id2 IDRecord - err = zson.UnmarshalZNG(zctx, arena, rec, &id2) + u := zson.NewZNGUnmarshaler() + u.Bind(IDRecord{}, ID{}) + err = zson.UnmarshalZNG(rec, &id2) require.NoError(t, err) assert.Equal(t, id, id2) b2 := BytesRecord{B: nil} - m = zson.NewZNGMarshalerWithContext(zctx) - rec, err = m.Marshal(arena, b2) + m = zson.NewZNGMarshaler() + rec, err = m.Marshal(b2) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, "{B:null(bytes)}", zson.FormatValue(rec)) s := SliceRecord{S: nil} - m = zson.NewZNGMarshalerWithContext(zctx) - rec, err = m.Marshal(arena, s) + m = zson.NewZNGMarshaler() + rec, err = m.Marshal(s) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, "{S:null([bytes])}", zson.FormatValue(rec)) @@ -154,13 +152,6 @@ type RecordWithInterfaceSlice struct { func TestMixedTypeArrayInsideRecord(t *testing.T) { t.Skip("see issue #4012") - - arena := zed.NewArena() - defer arena.Unref() - - m := zson.NewZNGMarshaler() - m.Decorate(zson.StyleSimple) - x := &RecordWithInterfaceSlice{ X: "hello", S: []Thing{ @@ -168,12 +159,15 @@ func TestMixedTypeArrayInsideRecord(t *testing.T) { &Animal{"blue"}, }, } - zv, err := m.Marshal(arena, x) + m := zson.NewZNGMarshaler() + m.Decorate(zson.StyleSimple) + + zv, err := m.Marshal(x) require.NoError(t, err) var buffer bytes.Buffer writer := zngio.NewWriter(zio.NopCloser(&buffer)) - recExpected := arena.New(zv.Type(), zv.Bytes()) + recExpected := zed.NewValue(zv.Type(), zv.Bytes()) writer.Write(recExpected) writer.Close() @@ -214,10 +208,6 @@ type MessageThing struct { func TestMixedTypeArrayOfStructWithInterface(t *testing.T) { t.Skip("see issue #4012") - - arena := zed.NewArena() - defer arena.Unref() - input := []MessageThing{ { Message: "hello", @@ -231,12 +221,12 @@ func TestMixedTypeArrayOfStructWithInterface(t *testing.T) { m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) - zv, err := m.Marshal(arena, input) + zv, err := m.Marshal(input) require.NoError(t, err) var buffer bytes.Buffer writer := zngio.NewWriter(zio.NopCloser(&buffer)) - recExpected := arena.New(zv.Type(), zv.Bytes()) + recExpected := zed.NewValue(zv.Type(), zv.Bytes()) writer.Write(recExpected) writer.Close() @@ -264,10 +254,9 @@ type Foo struct { } func TestUnexported(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() f := &Foo{1, 2} - _, err := zson.MarshalZNG(zed.NewContext(), arena, f) + m := zson.NewZNGMarshaler() + _, err := m.Marshal(f) require.NoError(t, err) } @@ -277,46 +266,39 @@ type ZNGValueField struct { } func TestZNGValueField(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - // Include a Zed int64 inside a Go struct as a zed.Value field. zngValueField := &ZNGValueField{ Name: "test1", Field: zed.NewInt64(123), } - m := zson.NewZNGMarshalerWithContext(zctx) + m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) - zv, err := m.Marshal(arena, zngValueField) + zv, err := m.Marshal(zngValueField) require.NoError(t, err) assert.Equal(t, `{Name:"test1",field:123}(=ZNGValueField)`, zson.FormatValue(zv)) u := zson.NewZNGUnmarshaler() - u.SetContext(zctx, arena) var out ZNGValueField err = u.Unmarshal(zv, &out) require.NoError(t, err) assert.Equal(t, zngValueField.Name, out.Name) assert.True(t, zngValueField.Field.Equal(out.Field)) // Include a Zed record inside a Go struct in a zed.Value field. - zv2, err := zson.ParseValue(zctx, arena, `{s:"foo",a:[1,2,3]}`) + zv2, err := zson.ParseValue(zed.NewContext(), `{s:"foo",a:[1,2,3]}`) require.NoError(t, err) zngValueField2 := &ZNGValueField{ Name: "test2", Field: zv2, } - m2 := zson.NewZNGMarshalerWithContext(zctx) + m2 := zson.NewZNGMarshaler() m2.Decorate(zson.StyleSimple) - zv3, err := m2.Marshal(arena, zngValueField2) + zv3, err := m2.Marshal(zngValueField2) require.NoError(t, err) assert.Equal(t, `{Name:"test2",field:{s:"foo",a:[1,2,3]}}(=ZNGValueField)`, zson.FormatValue(zv3)) u2 := zson.NewZNGUnmarshaler() - u2.SetContext(zctx, arena) var out2 ZNGValueField err = u2.Unmarshal(zv3, &out2) require.NoError(t, err) - assert.Equal(t, zngValueField2.Name, out2.Name) - assert.True(t, zngValueField2.Field.Equal(out2.Field)) + assert.Equal(t, *zngValueField2, out2) } func TestJSONFieldTag(t *testing.T) { @@ -440,9 +422,7 @@ func TestRecordWithMixedTypeNamedArrayElems(t *testing.T) { } m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) - arena := zed.NewArena() - defer arena.Unref() - val, err := m.Marshal(arena, in) + val, err := m.Marshal(in) require.NoError(t, err) u := zson.NewZNGUnmarshaler() u.Bind(Record{}, Array{}, Primitive{}) @@ -464,14 +444,11 @@ func TestInterfaceWithConcreteEmptyValue(t *testing.T) { func TestZedType(t *testing.T) { zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - u := zson.NewUnmarshaler() var typ zed.Type err := u.Unmarshal(``, &typ) assert.EqualError(t, err, `cannot unmarshal type value without type context`) - u.SetContext(zctx, arena) + u.SetContext(zctx) err = u.Unmarshal(``, &typ) require.NoError(t, err) assert.Equal(t, zed.TypeString, typ) diff --git a/zson/marshal_zng_test.go b/zson/marshal_zng_test.go index 7227b2e9ad..8980e92398 100644 --- a/zson/marshal_zng_test.go +++ b/zson/marshal_zng_test.go @@ -19,27 +19,23 @@ import ( ) func boomerang(t *testing.T, in interface{}, out interface{}) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - rec, err := zson.MarshalZNG(zctx, arena, in) + rec, err := zson.NewZNGMarshaler().Marshal(in) require.NoError(t, err) var buf bytes.Buffer zw := zngio.NewWriter(zio.NopCloser(&buf)) err = zw.Write(rec) require.NoError(t, err) require.NoError(t, zw.Close()) + zctx := zed.NewContext() zr := zngio.NewReader(zctx, &buf) defer zr.Close() val, err := zr.Read() require.NoError(t, err) - err = zson.UnmarshalZNG(zctx, arena, *val, out) + err = zson.UnmarshalZNG(*val, out) require.NoError(t, err) } func TestMarshalZNG(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() type S2 struct { Field2 string `zed:"f2"` Field3 int @@ -49,7 +45,7 @@ func TestMarshalZNG(t *testing.T) { Sub1 S2 PField1 *bool } - rec, err := zson.MarshalZNG(zed.NewContext(), arena, S1{ + rec, err := zson.NewZNGMarshaler().Marshal(S1{ Field1: "value1", Sub1: S2{ Field2: "value2", @@ -91,26 +87,24 @@ type ZNGThings struct { } func TestMarshalSlice(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) s := []ZNGThing{{"hello", 123}, {"world", 0}} r := ZNGThings{s} - rec, err := m.Marshal(arena, r) + rec, err := m.Marshal(r) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, `{Things:[{a:"hello",B:123}(=ZNGThing),{a:"world",B:0}(ZNGThing)]}(=ZNGThings)`, zson.FormatValue(rec)) empty := []ZNGThing{} r2 := ZNGThings{empty} - rec2, err := m.Marshal(arena, r2) + rec2, err := m.Marshal(r2) require.NoError(t, err) require.NotNil(t, rec2) assert.Equal(t, "{Things:[]([ZNGThing={a:string,B:int64}])}(=ZNGThings)", zson.FormatValue(rec2)) - rec3, err := m.Marshal(arena, ZNGThings{nil}) + rec3, err := m.Marshal(ZNGThings{nil}) require.NoError(t, err) require.NotNil(t, rec3) assert.Equal(t, "{Things:null([ZNGThing={a:string,B:int64}])}(=ZNGThings)", zson.FormatValue(rec3)) @@ -154,27 +148,21 @@ type TestIP struct { } func TestIPType(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - s := TestIP{Addr: netip.MustParseAddr("192.168.1.1")} - rec, err := zson.MarshalZNG(zctx, arena, s) + zctx := zed.NewContext() + m := zson.NewZNGMarshalerWithContext(zctx) + rec, err := m.Marshal(s) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, "{Addr:192.168.1.1}", zson.FormatValue(rec)) var tip TestIP - err = zson.UnmarshalZNG(zctx, arena, rec, &tip) + err = zson.UnmarshalZNG(rec, &tip) require.NoError(t, err) require.Equal(t, s, tip) } func TestUnmarshalRecord(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - type T3 struct { T3f1 int32 T3f2 float32 @@ -189,16 +177,16 @@ func TestUnmarshalRecord(t *testing.T) { v1 := T1{ T1f1: &T2{T2f1: T3{T3f1: 1, T3f2: 1.0}, T2f2: "t2f2-string1"}, } - rec, err := zson.MarshalZNG(zctx, arena, v1) + rec, err := zson.NewZNGMarshaler().Marshal(v1) require.NoError(t, err) require.NotNil(t, rec) const expected = `{top:{T2f1:{T3f1:1(int32),T3f2:1.(float32)},T2f2:"t2f2-string1"}}` require.Equal(t, expected, zson.FormatValue(rec)) - val := zson.MustParseValue(zctx, arena, expected) + val := zson.MustParseValue(zed.NewContext(), expected) var v2 T1 - err = zson.UnmarshalZNG(zctx, arena, val, &v2) + err = zson.UnmarshalZNG(val, &v2) require.NoError(t, err) require.Equal(t, v1, v2) @@ -206,7 +194,7 @@ func TestUnmarshalRecord(t *testing.T) { T4f1 *T2 `zed:"top"` } var v3 *T4 - err = zson.UnmarshalZNG(zctx, arena, rec, &v3) + err = zson.UnmarshalZNG(rec, &v3) require.NoError(t, err) require.NotNil(t, v3) require.NotNil(t, v3.T4f1) @@ -214,42 +202,39 @@ func TestUnmarshalRecord(t *testing.T) { } func TestUnmarshalNull(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() t.Run("slice", func(t *testing.T) { slice := []int{1} - require.NoError(t, zson.UnmarshalZNG(zctx, arena, zed.Null, &slice)) + require.NoError(t, zson.UnmarshalZNG(zed.Null, &slice)) assert.Nil(t, slice) slice = []int{1} - assert.EqualError(t, zson.UnmarshalZNG(zctx, arena, zed.NullInt64, &slice), `unmarshaling type "int64": not an array`) + assert.EqualError(t, zson.UnmarshalZNG(zed.NullInt64, &slice), `unmarshaling type "int64": not an array`) slice = []int{1} - v := zson.MustParseValue(zctx, arena, "null([int64])") - require.NoError(t, zson.UnmarshalZNG(zctx, arena, v, &slice)) + v := zson.MustParseValue(zed.NewContext(), "null([int64])") + require.NoError(t, zson.UnmarshalZNG(v, &slice)) assert.Nil(t, slice) - v = zson.MustParseValue(zctx, arena, "null(bytes)") + v = zson.MustParseValue(zed.NewContext(), "null(bytes)") buf := []byte("testing") - require.NoError(t, zson.UnmarshalZNG(zctx, arena, v, &buf)) + require.NoError(t, zson.UnmarshalZNG(v, &buf)) assert.Nil(t, buf) }) t.Run("primitive", func(t *testing.T) { integer := -1 - require.NoError(t, zson.UnmarshalZNG(zctx, arena, zed.Null, &integer)) + require.NoError(t, zson.UnmarshalZNG(zed.Null, &integer)) assert.Equal(t, integer, 0) intptr := &integer - require.NoError(t, zson.UnmarshalZNG(zctx, arena, zed.Null, &intptr)) + require.NoError(t, zson.UnmarshalZNG(zed.Null, &intptr)) assert.Nil(t, intptr) - assert.EqualError(t, zson.UnmarshalZNG(zctx, arena, zed.NullIP, &intptr), "incompatible type translation: zng type ip go type int go kind int") + assert.EqualError(t, zson.UnmarshalZNG(zed.NullIP, &intptr), "incompatible type translation: zng type ip go type int go kind int") }) t.Run("map", func(t *testing.T) { m := map[string]string{"key": "value"} - require.NoError(t, zson.UnmarshalZNG(zctx, arena, zed.Null, &m)) + require.NoError(t, zson.UnmarshalZNG(zed.Null, &m)) assert.Nil(t, m) - val := zson.MustParseValue(zctx, arena, "null({foo:int64})") - require.EqualError(t, zson.UnmarshalZNG(zctx, arena, val, &m), "not a map") + val := zson.MustParseValue(zed.NewContext(), "null({foo:int64})") + require.EqualError(t, zson.UnmarshalZNG(val, &m), "not a map") m = map[string]string{"key": "value"} - val = zson.MustParseValue(zctx, arena, "null(|{string:string}|)") - require.NoError(t, zson.UnmarshalZNG(zctx, arena, val, &m)) + val = zson.MustParseValue(zed.NewContext(), "null(|{string:string}|)") + require.NoError(t, zson.UnmarshalZNG(val, &m)) assert.Nil(t, m) }) t.Run("struct", func(t *testing.T) { @@ -259,38 +244,35 @@ func TestUnmarshalNull(t *testing.T) { var obj struct { Test *testobj `zed:"test"` } - val := zson.MustParseValue(zctx, arena, "{test:null({Val:int64})}") - require.NoError(t, zson.UnmarshalZNG(zctx, arena, val, &obj)) + val := zson.MustParseValue(zed.NewContext(), "{test:null({Val:int64})}") + require.NoError(t, zson.UnmarshalZNG(val, &obj)) require.Nil(t, obj.Test) - val = zson.MustParseValue(zctx, arena, "{test:null(ip)}") - require.EqualError(t, zson.UnmarshalZNG(zctx, arena, val, &obj), `cannot unmarshal Zed value "null(ip)" into Go struct`) + val = zson.MustParseValue(zed.NewContext(), "{test:null(ip)}") + require.EqualError(t, zson.UnmarshalZNG(val, &obj), `cannot unmarshal Zed value "null(ip)" into Go struct`) var slice struct { Test []string `zed:"test"` } slice.Test = []string{"1"} - val = zson.MustParseValue(zctx, arena, "{test:null}") - require.NoError(t, zson.UnmarshalZNG(zctx, arena, val, &slice)) + val = zson.MustParseValue(zed.NewContext(), "{test:null}") + require.NoError(t, zson.UnmarshalZNG(val, &slice)) require.Nil(t, slice.Test) }) } func TestUnmarshalSlice(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - type T1 struct { T1f1 []bool } v1 := T1{ T1f1: []bool{true, false, true}, } - rec, err := zson.MarshalZNG(zctx, arena, v1) + zctx := zed.NewContext() + rec, err := zson.NewZNGMarshalerWithContext(zctx).Marshal(v1) require.NoError(t, err) require.NotNil(t, rec) var v2 T1 - err = zson.UnmarshalZNG(zctx, arena, rec, &v2) + err = zson.UnmarshalZNG(rec, &v2) require.NoError(t, err) require.Equal(t, v1, v2) @@ -301,12 +283,13 @@ func TestUnmarshalSlice(t *testing.T) { v3 := T2{ Field1: []*int{intp(1), intp(2)}, } - rec, err = zson.MarshalZNG(zctx, arena, v3) + zctx = zed.NewContext() + rec, err = zson.NewZNGMarshalerWithContext(zctx).Marshal(v3) require.NoError(t, err) require.NotNil(t, rec) var v4 T2 - err = zson.UnmarshalZNG(zctx, arena, rec, &v4) + err = zson.UnmarshalZNG(rec, &v4) require.NoError(t, err) require.Equal(t, v1, v2) } @@ -331,33 +314,25 @@ func (m *testMarshaler) UnmarshalZNG(mc *zson.UnmarshalZNGContext, val zed.Value } func TestMarshalInterface(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - type rectype struct { M1 *testMarshaler M2 testMarshaler } m1 := testMarshaler("m1") r1 := rectype{M1: &m1, M2: testMarshaler("m2")} - rec, err := zson.MarshalZNG(zed.NewContext(), arena, r1) + rec, err := zson.NewZNGMarshaler().Marshal(r1) require.NoError(t, err) require.NotNil(t, rec) assert.Equal(t, `{M1:"marshal-m1",M2:"marshal-m2"}`, zson.FormatValue(rec)) var r2 rectype - err = zson.UnmarshalZNG(zctx, arena, rec, &r2) + err = zson.UnmarshalZNG(rec, &r2) require.NoError(t, err) assert.Equal(t, "m1", string(*r2.M1)) assert.Equal(t, "m2", string(r2.M2)) } func TestMarshalArray(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - type rectype struct { A1 [2]int8 A2 *[2]string @@ -365,14 +340,14 @@ func TestMarshalArray(t *testing.T) { } a2 := &[2]string{"foo", "bar"} r1 := rectype{A1: [2]int8{1, 2}, A2: a2} // A3 left as nil - rec, err := zson.MarshalZNG(zctx, arena, r1) + rec, err := zson.NewZNGMarshaler().Marshal(r1) require.NoError(t, err) require.NotNil(t, rec) const expected = `{A1:[1(int8),2(int8)],A2:["foo","bar"],A3:null([bytes])}` assert.Equal(t, expected, zson.FormatValue(rec)) var r2 rectype - err = zson.UnmarshalZNG(zctx, arena, rec, &r2) + err = zson.UnmarshalZNG(rec, &r2) require.NoError(t, err) assert.Equal(t, r1.A1, r2.A1) assert.Equal(t, *r2.A2, *r2.A2) @@ -380,10 +355,6 @@ func TestMarshalArray(t *testing.T) { } func TestNumbers(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - type rectype struct { I int I8 int8 @@ -414,28 +385,25 @@ func TestNumbers(t *testing.T) { F32: math.MaxFloat32, F64: math.MaxFloat64, } - rec, err := zson.MarshalZNG(zctx, arena, r1) + rec, err := zson.NewZNGMarshaler().Marshal(r1) require.NoError(t, err) require.NotNil(t, rec) const expected = "{I:-9223372036854775808,I8:-128(int8),I16:-32768(int16),I32:-2147483648(int32),I64:-9223372036854775808,U:18446744073709551615(uint64),UI8:255(uint8),UI16:65535(uint16),UI32:4294967295(uint32),UI64:18446744073709551615(uint64),F16:65504.(float16),F32:3.4028235e+38(float32),F64:1.7976931348623157e+308}" assert.Equal(t, expected, zson.FormatValue(rec)) var r2 rectype - err = zson.UnmarshalZNG(zctx, arena, rec, &r2) + err = zson.UnmarshalZNG(rec, &r2) require.NoError(t, err) assert.Equal(t, r1, r2) } func TestCustomRecord(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - vals := []interface{}{ ZNGThing{"hello", 123}, 99, } m := zson.NewZNGMarshaler() - rec, err := m.MarshalCustom(arena, []string{"foo", "bar"}, vals) + rec, err := m.MarshalCustom([]string{"foo", "bar"}, vals) require.NoError(t, err) assert.Equal(t, `{foo:{a:"hello",B:123},bar:99}`, zson.FormatValue(rec)) @@ -443,7 +411,7 @@ func TestCustomRecord(t *testing.T) { ZNGThing{"hello", 123}, nil, } - rec, err = m.MarshalCustom(arena, []string{"foo", "bar"}, vals) + rec, err = m.MarshalCustom([]string{"foo", "bar"}, vals) require.NoError(t, err) assert.Equal(t, `{foo:{a:"hello",B:123},bar:null}`, zson.FormatValue(rec)) } @@ -472,41 +440,35 @@ func Make(which int) ThingaMaBob { type Rolls []int func TestInterfaceZNGMarshal(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - t1 := Make(2) m := zson.NewZNGMarshaler() m.Decorate(zson.StylePackage) - zv, err := m.Marshal(arena, t1) + zv, err := m.Marshal(t1) require.NoError(t, err) assert.Equal(t, "zson_test.ThingTwo={c:string}", zson.String(zv.Type())) m.Decorate(zson.StyleSimple) rolls := Rolls{1, 2, 3} - zv, err = m.Marshal(arena, rolls) + zv, err = m.Marshal(rolls) require.NoError(t, err) assert.Equal(t, "Rolls=[int64]", zson.String(zv.Type())) m.Decorate(zson.StyleFull) - zv, err = m.Marshal(arena, rolls) + zv, err = m.Marshal(rolls) require.NoError(t, err) assert.Equal(t, `"github.com/brimdata/zed/zson_test.Rolls"=[int64]`, zson.String(zv.Type())) plain := []int32{1, 2, 3} - zv, err = m.Marshal(arena, plain) + zv, err = m.Marshal(plain) require.NoError(t, err) assert.Equal(t, "[int32]", zson.String(zv.Type())) } func TestInterfaceUnmarshal(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - t1 := Make(1) m := zson.NewZNGMarshaler() m.Decorate(zson.StylePackage) - zv, err := m.Marshal(arena, t1) + zv, err := m.Marshal(t1) require.NoError(t, err) assert.Equal(t, "zson_test.ZNGThing={a:string,B:int64}", zson.String(zv.Type())) @@ -533,16 +495,13 @@ func TestInterfaceUnmarshal(t *testing.T) { } func TestBindings(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - t1 := Make(1) m := zson.NewZNGMarshaler() m.NamedBindings([]zson.Binding{ {"SpecialThingOne", &ZNGThing{}}, {"SpecialThingTwo", &ThingTwo{}}, }) - zv, err := m.Marshal(arena, t1) + zv, err := m.Marshal(t1) require.NoError(t, err) assert.Equal(t, "SpecialThingOne={a:string,B:int64}", zson.String(zv.Type())) @@ -559,23 +518,19 @@ func TestBindings(t *testing.T) { } func TestEmptyInterface(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - - zv, err := zson.MarshalZNG(zctx, arena, int8(123)) + zv, err := zson.MarshalZNG(int8(123)) require.NoError(t, err) assert.Equal(t, "int8", zson.String(zv.Type())) var v interface{} - err = zson.UnmarshalZNG(zctx, arena, zv, &v) + err = zson.UnmarshalZNG(zv, &v) require.NoError(t, err) i, ok := v.(int8) assert.Equal(t, true, ok) assert.Equal(t, int8(123), i) var actual int8 - err = zson.UnmarshalZNG(zctx, arena, zv, &actual) + err = zson.UnmarshalZNG(zv, &actual) require.NoError(t, err) assert.Equal(t, int8(123), actual) } @@ -583,14 +538,11 @@ func TestEmptyInterface(t *testing.T) { type CustomInt8 int8 func TestNamedNormal(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - t1 := CustomInt8(88) m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) - zv, err := m.Marshal(arena, t1) + zv, err := m.Marshal(t1) require.NoError(t, err) assert.Equal(t, "CustomInt8=int8", zson.String(zv.Type())) @@ -618,15 +570,12 @@ type EmbeddedB struct { } func TestEmbeddedInterface(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - t1 := &EmbeddedA{ A: Make(1), } m := zson.NewZNGMarshaler() m.Decorate(zson.StyleSimple) - zv, err := m.Marshal(arena, t1) + zv, err := m.Marshal(t1) require.NoError(t, err) assert.Equal(t, "EmbeddedA={A:ZNGThing={a:string,B:int64}}", zson.String(zv.Type())) @@ -648,19 +597,15 @@ func TestEmbeddedInterface(t *testing.T) { } func TestMultipleZedValues(t *testing.T) { - arena := zed.NewArena() - defer arena.Unref() - bytes := []byte("foo") u := zson.NewZNGUnmarshaler() - u.SetContext(zed.NewContext(), arena) var foo zed.Value - err := u.Unmarshal(arena.New(zed.TypeString, bytes), &foo) + err := u.Unmarshal(zed.NewValue(zed.TypeString, bytes), &foo) require.NoError(t, err) // clobber bytes slice copy(bytes, []byte("bar")) var bar zed.Value - err = u.Unmarshal(arena.New(zed.TypeString, bytes), &bar) + err = u.Unmarshal(zed.NewValue(zed.TypeString, bytes), &bar) require.NoError(t, err) assert.Equal(t, "foo", string(foo.Bytes())) assert.Equal(t, "bar", string(bar.Bytes())) @@ -669,13 +614,10 @@ func TestMultipleZedValues(t *testing.T) { func TestZedValues(t *testing.T) { test := func(t *testing.T, name, s string, v interface{}) { t.Run(name, func(t *testing.T) { - zctx := zed.NewContext() - arena := zed.NewArena() - defer arena.Unref() - val := zson.MustParseValue(zctx, arena, s) - err := zson.UnmarshalZNG(zctx, arena, val, v) + val := zson.MustParseValue(zed.NewContext(), s) + err := zson.UnmarshalZNG(val, v) require.NoError(t, err) - val, err = zson.MarshalZNG(zctx, arena, v) + val, err = zson.MarshalZNG(v) require.NoError(t, err) assert.Equal(t, s, zson.FormatValue(val)) }) diff --git a/zson/parser-values.go b/zson/parser-values.go index c1402cd0d7..e4a9c0a373 100644 --- a/zson/parser-values.go +++ b/zson/parser-values.go @@ -602,7 +602,7 @@ func (p *Parser) matchTypeValue() (*astzed.TypeValue, error) { }, nil } -func ParsePrimitive(arena *zed.Arena, typeText, valText string) (zed.Value, error) { +func ParsePrimitive(typeText, valText string) (zed.Value, error) { typ := zed.LookupPrimitive(typeText) if typ == nil { return zed.Null, fmt.Errorf("no such type: %s", typeText) @@ -612,5 +612,5 @@ func ParsePrimitive(arena *zed.Arena, typeText, valText string) (zed.Value, erro return zed.Null, err } it := b.Bytes().Iter() - return arena.New(typ, it.Next()), nil + return zed.NewValue(typ, it.Next()), nil } diff --git a/zson/typeval_test.go b/zson/typeval_test.go index b6a7efb330..59f568925f 100644 --- a/zson/typeval_test.go +++ b/zson/typeval_test.go @@ -10,12 +10,9 @@ import ( func TestTypeValue(t *testing.T) { const s = "{A:{B:int64},C:int32}" - zctx := zed.NewContext() - typ, err := zson.ParseType(zctx, s) + typ, err := zson.ParseType(zed.NewContext(), s) require.NoError(t, err) - arena := zed.NewArena() - defer arena.Unref() - tv := zctx.LookupTypeValue(arena, typ) + tv := zed.NewContext().LookupTypeValue(typ) require.Exactly(t, s, zson.FormatTypeValue(tv.Bytes())) } @@ -23,8 +20,6 @@ func TestTypeValueCrossContext(t *testing.T) { const s = "{A:{B:int64},C:int32}" typ, err := zson.ParseType(zed.NewContext(), s) require.NoError(t, err) - arena := zed.NewArena() - defer arena.Unref() - tv := zed.NewContext().LookupTypeValue(arena, typ) + tv := zed.NewContext().LookupTypeValue(typ) require.Exactly(t, s, zson.FormatTypeValue(tv.Bytes())) } diff --git a/zson/zson.go b/zson/zson.go index 7e8adffd6d..b85b05743d 100644 --- a/zson/zson.go +++ b/zson/zson.go @@ -66,29 +66,33 @@ func ParseType(zctx *zed.Context, zson string) (zed.Type, error) { return NewAnalyzer().convertType(zctx, ast) } -func ParseValue(zctx *zed.Context, arena *zed.Arena, zson string) (zed.Value, error) { +func ParseValue(zctx *zed.Context, zson string) (zed.Value, error) { zp := NewParser(strings.NewReader(zson)) ast, err := zp.ParseValue() if err != nil { return zed.Null, err } - return ParseValueFromAST(zctx, arena, ast) + val, err := NewAnalyzer().ConvertValue(zctx, ast) + if err != nil { + return zed.Null, err + } + return Build(zcode.NewBuilder(), val) } -func MustParseValue(zctx *zed.Context, arena *zed.Arena, zson string) zed.Value { - val, err := ParseValue(zctx, arena, zson) +func MustParseValue(zctx *zed.Context, zson string) zed.Value { + val, err := ParseValue(zctx, zson) if err != nil { panic(err) } return val } -func ParseValueFromAST(zctx *zed.Context, arena *zed.Arena, ast astzed.Value) (zed.Value, error) { +func ParseValueFromAST(zctx *zed.Context, ast astzed.Value) (zed.Value, error) { val, err := NewAnalyzer().ConvertValue(zctx, ast) if err != nil { return zed.Null, err } - return Build(arena, zcode.NewBuilder(), val) + return Build(zcode.NewBuilder(), val) } func TranslateType(zctx *zed.Context, astType astzed.Type) (zed.Type, error) { diff --git a/zson/zson_test.go b/zson/zson_test.go index b3073a455c..6421626dc3 100644 --- a/zson/zson_test.go +++ b/zson/zson_test.go @@ -51,11 +51,11 @@ func TestZSONBuilder(t *testing.T) { zctx := zed.NewContext() val, err := analyze(zctx, testFile) require.NoError(t, err) - arena := zed.NewArena() - zv, err := zson.Build(arena, zcode.NewBuilder(), val) + b := zcode.NewBuilder() + zv, err := zson.Build(b, val) require.NoError(t, err) - rec := arena.New(zv.Type().(*zed.TypeRecord), zv.Bytes()) - a := rec.Deref(arena, "a") + rec := zed.NewValue(zv.Type().(*zed.TypeRecord), zv.Bytes()) + a := rec.Deref("a") assert.Equal(t, `["1","2","3"]`, zson.String(a)) } @@ -72,11 +72,9 @@ func TestParseValueStringEscapeSequences(t *testing.T) { {` "\u0000\u000A\u000b" `, "\u0000\u000A\u000b"}, } for _, c := range cases { - arena := zed.NewArena() - val, err := zson.ParseValue(zed.NewContext(), arena, c.in) + val, err := zson.ParseValue(zed.NewContext(), c.in) assert.NoError(t, err) - assert.True(t, val.Equal(arena.NewString(c.expected)), "in %q", c.in) - arena.Unref() + assert.Equal(t, zed.NewString(c.expected), val, "in %q", c.in) } } @@ -104,7 +102,7 @@ func TestParseValueErrors(t *testing.T) { {` "\v" `, `parse error: string literal: illegal escape (\v)`}, } for _, c := range cases { - _, err := zson.ParseValue(zed.NewContext(), zed.NewArena(), c.in) + _, err := zson.ParseValue(zed.NewContext(), c.in) assert.EqualError(t, err, c.expectedError, "in: %q", c.in) } } diff --git a/ztest/ztest.go b/ztest/ztest.go index b4a3088b50..65915cd330 100644 --- a/ztest/ztest.go +++ b/ztest/ztest.go @@ -574,7 +574,7 @@ func runvec(zedProgram string, input string, outputFlags []string) (string, stri return "", "", err } defer cleanup() - object, err := cache.Fetch(context.Background(), zctx, uri, ksuid.Nil) + object, err := cache.Fetch(context.Background(), uri, ksuid.Nil) if err != nil { return "", "", err }