syncbase: syncQL: integrate syncQL with key value store

This change implements Exec on the store's database object.
Later, when it is ready, Exec will also be added to BatchRequest.

They query package interacts with the store via implementaitons
of query's Database, Table and KeyValueStream interfaces (these
implemenations reside, for now, in
x/ref/services/syncbase/server/nosql/database.go.  The query_db
package needed to be moved out of internal to accomodate this.

Unlike Scan, which returns all errors via the stream, exec
needs to return execution errors on the function itself
(so forr example, the client doesn't attempt to print
headers before iterating over the stream).

Note: Rhis change requires a resolution to the error wrapping
issue.  See:
https://vanadium-review.googlesource.com/#/c/12651/
but we might as well start the review for this change.

Change-Id: I76d7fcee1557d36094c4e66cfee4cb6c9f8eedca
diff --git a/v23/services/syncbase/nosql/service.vdl b/v23/services/syncbase/nosql/service.vdl
index bee1209..8803897 100644
--- a/v23/services/syncbase/nosql/service.vdl
+++ b/v23/services/syncbase/nosql/service.vdl
@@ -36,6 +36,11 @@
 	// ErrNotBoundToBatch.
 	Commit() error {access.Read}
 
+	// Exec executes a syncQL query and returns all results as specified by
+	// in the query's select clause.  The returned stream reads
+	// from a consistent snapshot taken at the time of the Exec RPC.
+	Exec(query string) stream<_, []any> error {access.Read}
+
 	// Abort notifies the server that any pending changes can be discarded.
 	// It is not strictly required, but it may allow the server to release locks
 	// or other resources sooner than if it was not called.
diff --git a/v23/services/syncbase/nosql/service.vdl.go b/v23/services/syncbase/nosql/service.vdl.go
index 15f6af7..5319420 100644
--- a/v23/services/syncbase/nosql/service.vdl.go
+++ b/v23/services/syncbase/nosql/service.vdl.go
@@ -478,6 +478,10 @@
 	// If this Database is not bound to a batch, Commit() will fail with
 	// ErrNotBoundToBatch.
 	Commit(*context.T, ...rpc.CallOpt) error
+	// Exec executes a syncQL query and returns all results as specified by
+	// in the query's select clause.  The returned stream reads
+	// from a consistent snapshot taken at the time of the Exec RPC.
+	Exec(ctx *context.T, query string, opts ...rpc.CallOpt) (DatabaseExecClientCall, error)
 	// Abort notifies the server that any pending changes can be discarded.
 	// It is not strictly required, but it may allow the server to release locks
 	// or other resources sooner than if it was not called.
@@ -524,11 +528,88 @@
 	return
 }
 
+func (c implDatabaseClientStub) Exec(ctx *context.T, i0 string, opts ...rpc.CallOpt) (ocall DatabaseExecClientCall, err error) {
+	var call rpc.ClientCall
+	if call, err = v23.GetClient(ctx).StartCall(ctx, c.name, "Exec", []interface{}{i0}, opts...); err != nil {
+		return
+	}
+	ocall = &implDatabaseExecClientCall{ClientCall: call}
+	return
+}
+
 func (c implDatabaseClientStub) Abort(ctx *context.T, opts ...rpc.CallOpt) (err error) {
 	err = v23.GetClient(ctx).Call(ctx, c.name, "Abort", nil, nil, opts...)
 	return
 }
 
+// DatabaseExecClientStream is the client stream for Database.Exec.
+type DatabaseExecClientStream interface {
+	// RecvStream returns the receiver side of the Database.Exec client stream.
+	RecvStream() interface {
+		// Advance stages an item so that it may be retrieved via Value.  Returns
+		// true iff there is an item to retrieve.  Advance must be called before
+		// Value is called.  May block if an item is not available.
+		Advance() bool
+		// Value returns the item that was staged by Advance.  May panic if Advance
+		// returned false or was not called.  Never blocks.
+		Value() []*vdl.Value
+		// Err returns any error encountered by Advance.  Never blocks.
+		Err() error
+	}
+}
+
+// DatabaseExecClientCall represents the call returned from Database.Exec.
+type DatabaseExecClientCall interface {
+	DatabaseExecClientStream
+	// Finish blocks until the server is done, and returns the positional return
+	// values for call.
+	//
+	// Finish returns immediately if the call has been canceled; depending on the
+	// timing the output could either be an error signaling cancelation, or the
+	// valid positional return values from the server.
+	//
+	// Calling Finish is mandatory for releasing stream resources, unless the call
+	// has been canceled or any of the other methods return an error.  Finish should
+	// be called at most once.
+	Finish() error
+}
+
+type implDatabaseExecClientCall struct {
+	rpc.ClientCall
+	valRecv []*vdl.Value
+	errRecv error
+}
+
+func (c *implDatabaseExecClientCall) RecvStream() interface {
+	Advance() bool
+	Value() []*vdl.Value
+	Err() error
+} {
+	return implDatabaseExecClientCallRecv{c}
+}
+
+type implDatabaseExecClientCallRecv struct {
+	c *implDatabaseExecClientCall
+}
+
+func (c implDatabaseExecClientCallRecv) Advance() bool {
+	c.c.errRecv = c.c.Recv(&c.c.valRecv)
+	return c.c.errRecv == nil
+}
+func (c implDatabaseExecClientCallRecv) Value() []*vdl.Value {
+	return c.c.valRecv
+}
+func (c implDatabaseExecClientCallRecv) Err() error {
+	if c.c.errRecv == io.EOF {
+		return nil
+	}
+	return c.c.errRecv
+}
+func (c *implDatabaseExecClientCall) Finish() (err error) {
+	err = c.ClientCall.Finish()
+	return
+}
+
 // DatabaseServerMethods is the interface a server writer
 // implements for Database.
 //
@@ -602,6 +683,10 @@
 	// If this Database is not bound to a batch, Commit() will fail with
 	// ErrNotBoundToBatch.
 	Commit(*context.T, rpc.ServerCall) error
+	// Exec executes a syncQL query and returns all results as specified by
+	// in the query's select clause.  The returned stream reads
+	// from a consistent snapshot taken at the time of the Exec RPC.
+	Exec(ctx *context.T, call DatabaseExecServerCall, query string) error
 	// Abort notifies the server that any pending changes can be discarded.
 	// It is not strictly required, but it may allow the server to release locks
 	// or other resources sooner than if it was not called.
@@ -612,9 +697,84 @@
 
 // DatabaseServerStubMethods is the server interface containing
 // Database methods, as expected by rpc.Server.
-// There is no difference between this interface and DatabaseServerMethods
-// since there are no streaming methods.
-type DatabaseServerStubMethods DatabaseServerMethods
+// The only difference between this interface and DatabaseServerMethods
+// is the streaming methods.
+type DatabaseServerStubMethods interface {
+	// Object provides access control for Vanadium objects.
+	//
+	// Vanadium services implementing dynamic access control would typically embed
+	// this interface and tag additional methods defined by the service with one of
+	// Admin, Read, Write, Resolve etc. For example, the VDL definition of the
+	// object would be:
+	//
+	//   package mypackage
+	//
+	//   import "v.io/v23/security/access"
+	//   import "v.io/v23/services/permissions"
+	//
+	//   type MyObject interface {
+	//     permissions.Object
+	//     MyRead() (string, error) {access.Read}
+	//     MyWrite(string) error    {access.Write}
+	//   }
+	//
+	// If the set of pre-defined tags is insufficient, services may define their
+	// own tag type and annotate all methods with this new type.
+	//
+	// Instead of embedding this Object interface, define SetPermissions and
+	// GetPermissions in their own interface. Authorization policies will typically
+	// respect annotations of a single type. For example, the VDL definition of an
+	// object would be:
+	//
+	//  package mypackage
+	//
+	//  import "v.io/v23/security/access"
+	//
+	//  type MyTag string
+	//
+	//  const (
+	//    Blue = MyTag("Blue")
+	//    Red  = MyTag("Red")
+	//  )
+	//
+	//  type MyObject interface {
+	//    MyMethod() (string, error) {Blue}
+	//
+	//    // Allow clients to change access via the access.Object interface:
+	//    SetPermissions(perms access.Permissions, version string) error         {Red}
+	//    GetPermissions() (perms access.Permissions, version string, err error) {Blue}
+	//  }
+	permissions.ObjectServerStubMethods
+	// SyncGroupManager is the interface for SyncGroup operations.
+	// TODO(hpucha): Add blessings to create/join and add a refresh method.
+	SyncGroupManagerServerStubMethods
+	// Create creates this Database.
+	// If perms is nil, we inherit (copy) the App perms.
+	// Create requires the caller to have Write permission at the App.
+	Create(ctx *context.T, call rpc.ServerCall, perms access.Permissions) error
+	// Delete deletes this Database.
+	Delete(*context.T, rpc.ServerCall) error
+	// BeginBatch creates a new batch. It returns an App-relative name for a
+	// Database handle bound to this batch. If this Database is already bound to a
+	// batch, BeginBatch() will fail with ErrBoundToBatch.
+	//
+	// Concurrency semantics are documented in model.go.
+	BeginBatch(ctx *context.T, call rpc.ServerCall, bo BatchOptions) (string, error)
+	// Commit persists the pending changes to the database.
+	// If this Database is not bound to a batch, Commit() will fail with
+	// ErrNotBoundToBatch.
+	Commit(*context.T, rpc.ServerCall) error
+	// Exec executes a syncQL query and returns all results as specified by
+	// in the query's select clause.  The returned stream reads
+	// from a consistent snapshot taken at the time of the Exec RPC.
+	Exec(ctx *context.T, call *DatabaseExecServerCallStub, query string) error
+	// Abort notifies the server that any pending changes can be discarded.
+	// It is not strictly required, but it may allow the server to release locks
+	// or other resources sooner than if it was not called.
+	// If this Database is not bound to a batch, Abort() will fail with
+	// ErrNotBoundToBatch.
+	Abort(*context.T, rpc.ServerCall) error
+}
 
 // DatabaseServerStub adds universal methods to DatabaseServerStubMethods.
 type DatabaseServerStub interface {
@@ -665,6 +825,10 @@
 	return s.impl.Commit(ctx, call)
 }
 
+func (s implDatabaseServerStub) Exec(ctx *context.T, call *DatabaseExecServerCallStub, i0 string) error {
+	return s.impl.Exec(ctx, call, i0)
+}
+
 func (s implDatabaseServerStub) Abort(ctx *context.T, call rpc.ServerCall) error {
 	return s.impl.Abort(ctx, call)
 }
@@ -720,6 +884,14 @@
 			Tags: []*vdl.Value{vdl.ValueOf(access.Tag("Read"))},
 		},
 		{
+			Name: "Exec",
+			Doc:  "// Exec executes a syncQL query and returns all results as specified by\n// in the query's select clause.  The returned stream reads\n// from a consistent snapshot taken at the time of the Exec RPC.",
+			InArgs: []rpc.ArgDesc{
+				{"query", ``}, // string
+			},
+			Tags: []*vdl.Value{vdl.ValueOf(access.Tag("Read"))},
+		},
+		{
 			Name: "Abort",
 			Doc:  "// Abort notifies the server that any pending changes can be discarded.\n// It is not strictly required, but it may allow the server to release locks\n// or other resources sooner than if it was not called.\n// If this Database is not bound to a batch, Abort() will fail with\n// ErrNotBoundToBatch.",
 			Tags: []*vdl.Value{vdl.ValueOf(access.Tag("Read"))},
@@ -727,6 +899,49 @@
 	},
 }
 
+// DatabaseExecServerStream is the server stream for Database.Exec.
+type DatabaseExecServerStream interface {
+	// SendStream returns the send side of the Database.Exec server stream.
+	SendStream() interface {
+		// Send places the item onto the output stream.  Returns errors encountered
+		// while sending.  Blocks if there is no buffer space; will unblock when
+		// buffer space is available.
+		Send(item []*vdl.Value) error
+	}
+}
+
+// DatabaseExecServerCall represents the context passed to Database.Exec.
+type DatabaseExecServerCall interface {
+	rpc.ServerCall
+	DatabaseExecServerStream
+}
+
+// DatabaseExecServerCallStub is a wrapper that converts rpc.StreamServerCall into
+// a typesafe stub that implements DatabaseExecServerCall.
+type DatabaseExecServerCallStub struct {
+	rpc.StreamServerCall
+}
+
+// Init initializes DatabaseExecServerCallStub from rpc.StreamServerCall.
+func (s *DatabaseExecServerCallStub) Init(call rpc.StreamServerCall) {
+	s.StreamServerCall = call
+}
+
+// SendStream returns the send side of the Database.Exec server stream.
+func (s *DatabaseExecServerCallStub) SendStream() interface {
+	Send(item []*vdl.Value) error
+} {
+	return implDatabaseExecServerCallSend{s}
+}
+
+type implDatabaseExecServerCallSend struct {
+	s *DatabaseExecServerCallStub
+}
+
+func (s implDatabaseExecServerCallSend) Send(item []*vdl.Value) error {
+	return s.s.Send(item)
+}
+
 // TableClientMethods is the client interface
 // containing Table methods.
 //
diff --git a/v23/syncbase/nosql/client_test.go b/v23/syncbase/nosql/client_test.go
index 87d0581..435ee50 100644
--- a/v23/syncbase/nosql/client_test.go
+++ b/v23/syncbase/nosql/client_test.go
@@ -10,8 +10,10 @@
 
 	"v.io/syncbase/v23/syncbase"
 	"v.io/syncbase/v23/syncbase/nosql"
+	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	tu "v.io/syncbase/v23/syncbase/testutil"
 	"v.io/v23/naming"
+	"v.io/v23/vdl"
 	"v.io/v23/verror"
 	_ "v.io/x/ref/runtime/factories/generic"
 )
@@ -52,6 +54,83 @@
 	tu.TestCreate(t, ctx, a)
 }
 
+// Tests that Database.Exec works as expected.
+// Note: More comprehensive client/server tests are in the exec_test
+// directory.  Also, exec is tested in its entirety in
+// v23/syncbase/nosql/internal/query/...
+func TestExec(t *testing.T) {
+	ctx, sName, cleanup := tu.SetupOrDie(nil)
+	defer cleanup()
+	a := tu.CreateApp(t, ctx, syncbase.NewService(sName), "a")
+	d := tu.CreateNoSQLDatabase(t, ctx, a, "d")
+	tb := tu.CreateTable(t, ctx, d, "tb")
+
+	foo := Foo{I: 4, S: "f"}
+	if err := tb.Put(ctx, "foo", foo); err != nil {
+		t.Fatalf("tb.Put() failed: %v", err)
+	}
+
+	bar := Bar{F: 0.5, S: "b"}
+	// NOTE: not best practice, but store bar as
+	// optional (by passing the address of bar to Put).
+	// This tests auto-dereferencing.
+	if err := tb.Put(ctx, "bar", &bar); err != nil {
+		t.Fatalf("tb.Put() failed: %v", err)
+	}
+
+	baz := Baz{Name: "John Doe", Active: true}
+	if err := tb.Put(ctx, "baz", baz); err != nil {
+		t.Fatalf("tb.Put() failed: %v", err)
+	}
+
+	tu.CheckExec(t, ctx, d, "select k, v.Name from tb where t = \"Baz\"",
+		[]string{"k", "v.Name"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("baz"), vdl.ValueOf(baz.Name)},
+		})
+
+	tu.CheckExec(t, ctx, d, "select k, v from tb",
+		[]string{"k", "v"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("bar"), vdl.ValueOf(bar)},
+			[]*vdl.Value{vdl.ValueOf("baz"), vdl.ValueOf(baz)},
+			[]*vdl.Value{vdl.ValueOf("foo"), vdl.ValueOf(foo)},
+		})
+
+	tu.CheckExec(t, ctx, d, "select k, v from tb where k like \"ba%\"",
+		[]string{"k", "v"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("bar"), vdl.ValueOf(bar)},
+			[]*vdl.Value{vdl.ValueOf("baz"), vdl.ValueOf(baz)},
+		})
+
+	tu.CheckExec(t, ctx, d, "select k, v from tb where v.Active = true",
+		[]string{"k", "v"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("baz"), vdl.ValueOf(baz)},
+		})
+
+	tu.CheckExec(t, ctx, d, "select k, v from tb where t = \"Bar\"",
+		[]string{"k", "v"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("bar"), vdl.ValueOf(bar)},
+		})
+
+	tu.CheckExec(t, ctx, d, "select k, v from tb where v.F = 0.5",
+		[]string{"k", "v"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("bar"), vdl.ValueOf(bar)},
+		})
+
+	tu.CheckExec(t, ctx, d, "select k, v from tb where t = \"Baz\"",
+		[]string{"k", "v"},
+		[][]*vdl.Value{
+			[]*vdl.Value{vdl.ValueOf("baz"), vdl.ValueOf(baz)},
+		})
+
+	tu.CheckExecError(t, ctx, d, "select k, v from foo", syncql.ErrTableCantAccess.ID)
+}
+
 // Tests that Database.Delete works as expected.
 func TestDatabaseDelete(t *testing.T) {
 	ctx, sName, cleanup := tu.SetupOrDie(nil)
@@ -116,6 +195,11 @@
 	S string
 }
 
+type Baz struct {
+	Name   string
+	Active bool
+}
+
 // Tests that Table.Scan works as expected.
 func TestTableScan(t *testing.T) {
 	ctx, sName, cleanup := tu.SetupOrDie(nil)
diff --git a/v23/syncbase/nosql/database.go b/v23/syncbase/nosql/database.go
index d778702..8346109 100644
--- a/v23/syncbase/nosql/database.go
+++ b/v23/syncbase/nosql/database.go
@@ -73,6 +73,30 @@
 	return wire.TableClient(naming.Join(d.fullName, relativeName)).Delete(ctx)
 }
 
+// Exec implements Database.Exec.
+func (d *database) Exec(ctx *context.T, query string) ([]string, ResultStream, error) {
+	ctx, cancel := context.WithCancel(ctx)
+	call, err := d.c.Exec(ctx, query)
+	if err != nil {
+		return nil, nil, err
+	}
+	resultStream := newResultStream(cancel, call)
+	// The first row contains headers, pull them off the stream
+	// and return them separately.
+	var headers []string
+	if !resultStream.Advance() {
+		if err = resultStream.Err(); err != nil {
+			// Since there was an error, can't get headers.
+			// Just return the error.
+			return nil, nil, err
+		}
+	}
+	for _, header := range resultStream.Result() {
+		headers = append(headers, header.RawString())
+	}
+	return headers, resultStream, nil
+}
+
 // BeginBatch implements Database.BeginBatch.
 func (d *database) BeginBatch(ctx *context.T, opts wire.BatchOptions) (BatchDatabase, error) {
 	relativeName, err := d.c.BeginBatch(ctx, opts)
diff --git a/v23/syncbase/nosql/exec_test/db_objects.vdl b/v23/syncbase/nosql/exec_test/db_objects.vdl
new file mode 100644
index 0000000..fb75d36
--- /dev/null
+++ b/v23/syncbase/nosql/exec_test/db_objects.vdl
@@ -0,0 +1,96 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package exectest
+
+import "time"
+
+type AddressInfo struct {
+	Street string
+	City   string
+	State  string
+	Zip    string
+}
+
+type CreditAgency enum {
+	Equifax
+	Experian
+	TransUnion
+}
+
+type ExperianRating enum {
+	Good
+	Bad
+}
+
+type EquifaxCreditReport struct {
+	Rating byte
+}
+
+type ExperianCreditReport struct {
+	Rating ExperianRating
+}
+
+type TransUnionCreditReport struct {
+	Rating int16
+}
+
+type AgencyReport union {
+	EquifaxReport    EquifaxCreditReport
+	ExperianReport   ExperianCreditReport
+	TransUnionReport TransUnionCreditReport
+}
+
+type CreditReport struct {
+	Agency CreditAgency
+	Report AgencyReport
+}
+
+type Customer struct {
+	Name    string
+	Id      int64
+	Active  bool
+	Address AddressInfo
+	Credit  CreditReport
+}
+
+type Invoice struct {
+	CustId      int64
+	InvoiceNum  int64
+	InvoiceDate time.Time
+	Amount      int64
+	ShipTo      AddressInfo
+}
+
+type Numbers struct {
+	B    byte
+	Ui16 uint16
+	Ui32 uint32
+	Ui64 uint64
+	I16  int16
+	I32  int32
+	I64  int64
+	F32  float32
+	F64  float64
+	C64  complex64
+	C128 complex128
+}
+
+type FooType struct {
+	Bar BarType
+}
+
+type BarType struct {
+	Baz BazType
+}
+
+type TitleOrValueType union {
+	Title string
+	Value int64
+}
+
+type BazType struct {
+	Name         string
+	TitleOrValue TitleOrValueType
+}
diff --git a/v23/syncbase/nosql/exec_test/db_objects.vdl.go b/v23/syncbase/nosql/exec_test/db_objects.vdl.go
new file mode 100644
index 0000000..321556c
--- /dev/null
+++ b/v23/syncbase/nosql/exec_test/db_objects.vdl.go
@@ -0,0 +1,340 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// This file was auto-generated by the vanadium vdl tool.
+// Source: db_objects.vdl
+
+package exectest
+
+import (
+	// VDL system imports
+	"fmt"
+	"v.io/v23/vdl"
+
+	// VDL user imports
+	"time"
+	_ "v.io/v23/vdlroot/time"
+)
+
+type AddressInfo struct {
+	Street string
+	City   string
+	State  string
+	Zip    string
+}
+
+func (AddressInfo) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.AddressInfo"`
+}) {
+}
+
+type CreditAgency int
+
+const (
+	CreditAgencyEquifax CreditAgency = iota
+	CreditAgencyExperian
+	CreditAgencyTransUnion
+)
+
+// CreditAgencyAll holds all labels for CreditAgency.
+var CreditAgencyAll = [...]CreditAgency{CreditAgencyEquifax, CreditAgencyExperian, CreditAgencyTransUnion}
+
+// CreditAgencyFromString creates a CreditAgency from a string label.
+func CreditAgencyFromString(label string) (x CreditAgency, err error) {
+	err = x.Set(label)
+	return
+}
+
+// Set assigns label to x.
+func (x *CreditAgency) Set(label string) error {
+	switch label {
+	case "Equifax", "equifax":
+		*x = CreditAgencyEquifax
+		return nil
+	case "Experian", "experian":
+		*x = CreditAgencyExperian
+		return nil
+	case "TransUnion", "transunion":
+		*x = CreditAgencyTransUnion
+		return nil
+	}
+	*x = -1
+	return fmt.Errorf("unknown label %q in exectest.CreditAgency", label)
+}
+
+// String returns the string label of x.
+func (x CreditAgency) String() string {
+	switch x {
+	case CreditAgencyEquifax:
+		return "Equifax"
+	case CreditAgencyExperian:
+		return "Experian"
+	case CreditAgencyTransUnion:
+		return "TransUnion"
+	}
+	return ""
+}
+
+func (CreditAgency) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.CreditAgency"`
+	Enum struct{ Equifax, Experian, TransUnion string }
+}) {
+}
+
+type ExperianRating int
+
+const (
+	ExperianRatingGood ExperianRating = iota
+	ExperianRatingBad
+)
+
+// ExperianRatingAll holds all labels for ExperianRating.
+var ExperianRatingAll = [...]ExperianRating{ExperianRatingGood, ExperianRatingBad}
+
+// ExperianRatingFromString creates a ExperianRating from a string label.
+func ExperianRatingFromString(label string) (x ExperianRating, err error) {
+	err = x.Set(label)
+	return
+}
+
+// Set assigns label to x.
+func (x *ExperianRating) Set(label string) error {
+	switch label {
+	case "Good", "good":
+		*x = ExperianRatingGood
+		return nil
+	case "Bad", "bad":
+		*x = ExperianRatingBad
+		return nil
+	}
+	*x = -1
+	return fmt.Errorf("unknown label %q in exectest.ExperianRating", label)
+}
+
+// String returns the string label of x.
+func (x ExperianRating) String() string {
+	switch x {
+	case ExperianRatingGood:
+		return "Good"
+	case ExperianRatingBad:
+		return "Bad"
+	}
+	return ""
+}
+
+func (ExperianRating) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.ExperianRating"`
+	Enum struct{ Good, Bad string }
+}) {
+}
+
+type EquifaxCreditReport struct {
+	Rating byte
+}
+
+func (EquifaxCreditReport) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.EquifaxCreditReport"`
+}) {
+}
+
+type ExperianCreditReport struct {
+	Rating ExperianRating
+}
+
+func (ExperianCreditReport) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.ExperianCreditReport"`
+}) {
+}
+
+type TransUnionCreditReport struct {
+	Rating int16
+}
+
+func (TransUnionCreditReport) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.TransUnionCreditReport"`
+}) {
+}
+
+type (
+	// AgencyReport represents any single field of the AgencyReport union type.
+	AgencyReport interface {
+		// Index returns the field index.
+		Index() int
+		// Interface returns the field value as an interface.
+		Interface() interface{}
+		// Name returns the field name.
+		Name() string
+		// __VDLReflect describes the AgencyReport union type.
+		__VDLReflect(__AgencyReportReflect)
+	}
+	// AgencyReportEquifaxReport represents field EquifaxReport of the AgencyReport union type.
+	AgencyReportEquifaxReport struct{ Value EquifaxCreditReport }
+	// AgencyReportExperianReport represents field ExperianReport of the AgencyReport union type.
+	AgencyReportExperianReport struct{ Value ExperianCreditReport }
+	// AgencyReportTransUnionReport represents field TransUnionReport of the AgencyReport union type.
+	AgencyReportTransUnionReport struct{ Value TransUnionCreditReport }
+	// __AgencyReportReflect describes the AgencyReport union type.
+	__AgencyReportReflect struct {
+		Name  string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.AgencyReport"`
+		Type  AgencyReport
+		Union struct {
+			EquifaxReport    AgencyReportEquifaxReport
+			ExperianReport   AgencyReportExperianReport
+			TransUnionReport AgencyReportTransUnionReport
+		}
+	}
+)
+
+func (x AgencyReportEquifaxReport) Index() int                         { return 0 }
+func (x AgencyReportEquifaxReport) Interface() interface{}             { return x.Value }
+func (x AgencyReportEquifaxReport) Name() string                       { return "EquifaxReport" }
+func (x AgencyReportEquifaxReport) __VDLReflect(__AgencyReportReflect) {}
+
+func (x AgencyReportExperianReport) Index() int                         { return 1 }
+func (x AgencyReportExperianReport) Interface() interface{}             { return x.Value }
+func (x AgencyReportExperianReport) Name() string                       { return "ExperianReport" }
+func (x AgencyReportExperianReport) __VDLReflect(__AgencyReportReflect) {}
+
+func (x AgencyReportTransUnionReport) Index() int                         { return 2 }
+func (x AgencyReportTransUnionReport) Interface() interface{}             { return x.Value }
+func (x AgencyReportTransUnionReport) Name() string                       { return "TransUnionReport" }
+func (x AgencyReportTransUnionReport) __VDLReflect(__AgencyReportReflect) {}
+
+type CreditReport struct {
+	Agency CreditAgency
+	Report AgencyReport
+}
+
+func (CreditReport) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.CreditReport"`
+}) {
+}
+
+type Customer struct {
+	Name    string
+	Id      int64
+	Active  bool
+	Address AddressInfo
+	Credit  CreditReport
+}
+
+func (Customer) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.Customer"`
+}) {
+}
+
+type Invoice struct {
+	CustId      int64
+	InvoiceNum  int64
+	InvoiceDate time.Time
+	Amount      int64
+	ShipTo      AddressInfo
+}
+
+func (Invoice) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.Invoice"`
+}) {
+}
+
+type Numbers struct {
+	B    byte
+	Ui16 uint16
+	Ui32 uint32
+	Ui64 uint64
+	I16  int16
+	I32  int32
+	I64  int64
+	F32  float32
+	F64  float64
+	C64  complex64
+	C128 complex128
+}
+
+func (Numbers) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.Numbers"`
+}) {
+}
+
+type FooType struct {
+	Bar BarType
+}
+
+func (FooType) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.FooType"`
+}) {
+}
+
+type BarType struct {
+	Baz BazType
+}
+
+func (BarType) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.BarType"`
+}) {
+}
+
+type (
+	// TitleOrValueType represents any single field of the TitleOrValueType union type.
+	TitleOrValueType interface {
+		// Index returns the field index.
+		Index() int
+		// Interface returns the field value as an interface.
+		Interface() interface{}
+		// Name returns the field name.
+		Name() string
+		// __VDLReflect describes the TitleOrValueType union type.
+		__VDLReflect(__TitleOrValueTypeReflect)
+	}
+	// TitleOrValueTypeTitle represents field Title of the TitleOrValueType union type.
+	TitleOrValueTypeTitle struct{ Value string }
+	// TitleOrValueTypeValue represents field Value of the TitleOrValueType union type.
+	TitleOrValueTypeValue struct{ Value int64 }
+	// __TitleOrValueTypeReflect describes the TitleOrValueType union type.
+	__TitleOrValueTypeReflect struct {
+		Name  string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.TitleOrValueType"`
+		Type  TitleOrValueType
+		Union struct {
+			Title TitleOrValueTypeTitle
+			Value TitleOrValueTypeValue
+		}
+	}
+)
+
+func (x TitleOrValueTypeTitle) Index() int                             { return 0 }
+func (x TitleOrValueTypeTitle) Interface() interface{}                 { return x.Value }
+func (x TitleOrValueTypeTitle) Name() string                           { return "Title" }
+func (x TitleOrValueTypeTitle) __VDLReflect(__TitleOrValueTypeReflect) {}
+
+func (x TitleOrValueTypeValue) Index() int                             { return 1 }
+func (x TitleOrValueTypeValue) Interface() interface{}                 { return x.Value }
+func (x TitleOrValueTypeValue) Name() string                           { return "Value" }
+func (x TitleOrValueTypeValue) __VDLReflect(__TitleOrValueTypeReflect) {}
+
+type BazType struct {
+	Name         string
+	TitleOrValue TitleOrValueType
+}
+
+func (BazType) __VDLReflect(struct {
+	Name string `vdl:"v.io/syncbase/v23/syncbase/nosql/exec_test.BazType"`
+}) {
+}
+
+func init() {
+	vdl.Register((*AddressInfo)(nil))
+	vdl.Register((*CreditAgency)(nil))
+	vdl.Register((*ExperianRating)(nil))
+	vdl.Register((*EquifaxCreditReport)(nil))
+	vdl.Register((*ExperianCreditReport)(nil))
+	vdl.Register((*TransUnionCreditReport)(nil))
+	vdl.Register((*AgencyReport)(nil))
+	vdl.Register((*CreditReport)(nil))
+	vdl.Register((*Customer)(nil))
+	vdl.Register((*Invoice)(nil))
+	vdl.Register((*Numbers)(nil))
+	vdl.Register((*FooType)(nil))
+	vdl.Register((*BarType)(nil))
+	vdl.Register((*TitleOrValueType)(nil))
+	vdl.Register((*BazType)(nil))
+}
diff --git a/v23/syncbase/nosql/exec_test/doc.go b/v23/syncbase/nosql/exec_test/doc.go
new file mode 100644
index 0000000..96025d1
--- /dev/null
+++ b/v23/syncbase/nosql/exec_test/doc.go
@@ -0,0 +1,6 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+// Package exectest contains tests for syncbase's Exec (syncQL) method.
+package exectest
diff --git a/v23/syncbase/nosql/exec_test/exec_test.go b/v23/syncbase/nosql/exec_test/exec_test.go
new file mode 100644
index 0000000..fdd63ca
--- /dev/null
+++ b/v23/syncbase/nosql/exec_test/exec_test.go
@@ -0,0 +1,948 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package exectest
+
+import (
+	"errors"
+	"reflect"
+	"strings"
+	"testing"
+	"time"
+
+	"v.io/syncbase/v23/syncbase"
+	"v.io/syncbase/v23/syncbase/nosql"
+	"v.io/syncbase/v23/syncbase/nosql/syncql"
+	tu "v.io/syncbase/v23/syncbase/testutil"
+	"v.io/v23/context"
+	"v.io/v23/vdl"
+	"v.io/v23/verror"
+	_ "v.io/x/ref/runtime/factories/generic"
+)
+
+var ctx *context.T
+var db nosql.Database
+var cleanup func()
+
+// In addition to populating store, populate these arrays to make
+// specifying the wanted values in the tests easier.
+type kv struct {
+	key   string
+	value *vdl.Value
+}
+
+var customerEntries []kv
+var numbersEntries []kv
+var fooEntries []kv
+
+var t2015 time.Time
+
+var t2015_04 time.Time
+var t2015_04_12 time.Time
+var t2015_04_12_22 time.Time
+var t2015_04_12_22_16 time.Time
+var t2015_04_12_22_16_06 time.Time
+
+var t2015_07 time.Time
+var t2015_07_01 time.Time
+var t2015_07_01_01_23_45 time.Time
+
+func setup(t *testing.T) {
+	var sName string
+	ctx, sName, cleanup = tu.SetupOrDie(nil)
+	a := tu.CreateApp(t, ctx, syncbase.NewService(sName), "a")
+	db = tu.CreateNoSQLDatabase(t, ctx, a, "db")
+	customerTable := tu.CreateTable(t, ctx, db, "Customer")
+	numbersTable := tu.CreateTable(t, ctx, db, "Numbers")
+	fooTable := tu.CreateTable(t, ctx, db, "Foo")
+
+	t20150122131101, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Jan 22 2015 13:11:01 -0800 PST")
+	t20150210161202, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Feb 10 2015 16:12:02 -0800 PST")
+	t20150311101303, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Mar 11 2015 10:13:03 -0700 PDT")
+	t20150317111404, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Mar 17 2015 11:14:04 -0700 PDT")
+	t20150317131505, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Mar 17 2015 13:15:05 -0700 PDT")
+	t20150412221606, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Apr 12 2015 22:16:06 -0700 PDT")
+	t20150413141707, _ := time.Parse("Jan 2 2006 15:04:05 -0700 MST", "Apr 13 2015 14:17:07 -0700 PDT")
+
+	t2015, _ = time.Parse("2006 MST", "2015 PST")
+
+	t2015_04, _ = time.Parse("Jan 2006 MST", "Apr 2015 PDT")
+	t2015_07, _ = time.Parse("Jan 2006 MST", "Jul 2015 PDT")
+
+	t2015_04_12, _ = time.Parse("Jan 2 2006 MST", "Apr 12 2015 PDT")
+	t2015_07_01, _ = time.Parse("Jan 2 2006 MST", "Jul 01 2015 PDT")
+
+	t2015_04_12_22, _ = time.Parse("Jan 2 2006 15 MST", "Apr 12 2015 22 PDT")
+	t2015_04_12_22_16, _ = time.Parse("Jan 2 2006 15:04 MST", "Apr 12 2015 22:16 PDT")
+	t2015_04_12_22_16_06, _ = time.Parse("Jan 2 2006 15:04:05 MST", "Apr 12 2015 22:16:06 PDT")
+	t2015_07_01_01_23_45, _ = time.Parse("Jan 2 2006 15:04:05 MST", "Jul 01 2015 01:23:45 PDT")
+
+	k := "001"
+	c := Customer{"John Smith", 1, true, AddressInfo{"1 Main St.", "Palo Alto", "CA", "94303"}, CreditReport{Agency: CreditAgencyEquifax, Report: AgencyReportEquifaxReport{EquifaxCreditReport{'A'}}}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(c)})
+	if err := customerTable.Put(ctx, k, c); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+	k = "001001"
+	i := Invoice{1, 1000, t20150122131101, 42, AddressInfo{"1 Main St.", "Palo Alto", "CA", "94303"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "001002"
+	i = Invoice{1, 1003, t20150210161202, 7, AddressInfo{"2 Main St.", "Palo Alto", "CA", "94303"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "001003"
+	i = Invoice{1, 1005, t20150311101303, 88, AddressInfo{"3 Main St.", "Palo Alto", "CA", "94303"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "002"
+	c = Customer{"Bat Masterson", 2, true, AddressInfo{"777 Any St.", "Collins", "IA", "50055"}, CreditReport{Agency: CreditAgencyTransUnion, Report: AgencyReportTransUnionReport{TransUnionCreditReport{80}}}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(c)})
+	if err := customerTable.Put(ctx, k, c); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "002001"
+	i = Invoice{2, 1001, t20150317111404, 166, AddressInfo{"777 Any St.", "collins", "IA", "50055"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "002002"
+	i = Invoice{2, 1002, t20150317131505, 243, AddressInfo{"888 Any St.", "collins", "IA", "50055"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "002003"
+	i = Invoice{2, 1004, t20150412221606, 787, AddressInfo{"999 Any St.", "collins", "IA", "50055"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "002004"
+	i = Invoice{2, 1006, t20150413141707, 88, AddressInfo{"101010 Any St.", "collins", "IA", "50055"}}
+	customerEntries = append(customerEntries, kv{k, vdl.ValueOf(i)})
+	if err := customerTable.Put(ctx, k, i); err != nil {
+		t.Fatalf("customerTable.Put() failed: %v", err)
+	}
+
+	k = "001"
+	n := Numbers{byte(12), uint16(1234), uint32(5678), uint64(999888777666), int16(9876), int32(876543), int64(128), float32(3.14159), float64(2.71828182846), complex64(123.0 + 7.0i), complex128(456.789 + 10.1112i)}
+	numbersEntries = append(numbersEntries, kv{k, vdl.ValueOf(n)})
+	if err := numbersTable.Put(ctx, k, n); err != nil {
+		t.Fatalf("numbersTable.Put() failed: %v", err)
+	}
+
+	k = "002"
+	n = Numbers{byte(9), uint16(99), uint32(999), uint64(9999999), int16(9), int32(99), int64(88), float32(1.41421356237), float64(1.73205080757), complex64(9.87 + 7.65i), complex128(4.32 + 1.0i)}
+	numbersEntries = append(numbersEntries, kv{k, vdl.ValueOf(n)})
+	if err := numbersTable.Put(ctx, k, n); err != nil {
+		t.Fatalf("numbersTable.Put() failed: %v", err)
+	}
+
+	k = "003"
+	n = Numbers{byte(210), uint16(210), uint32(210), uint64(210), int16(210), int32(210), int64(210), float32(210.0), float64(210.0), complex64(210.0 + 0.0i), complex128(210.0 + 0.0i)}
+	numbersEntries = append(numbersEntries, kv{k, vdl.ValueOf(n)})
+	if err := numbersTable.Put(ctx, k, n); err != nil {
+		t.Fatalf("numbersTable.Put() failed: %v", err)
+	}
+
+	k = "001"
+	f := FooType{BarType{BazType{"FooBarBaz", TitleOrValueTypeTitle{"Vice President"}}}}
+	fooEntries = append(fooEntries, kv{k, vdl.ValueOf(f)})
+	if err := fooTable.Put(ctx, k, f); err != nil {
+		t.Fatalf("fooTable.Put() failed: %v", err)
+	}
+
+	k = "002"
+	f = FooType{BarType{BazType{"BazBarFoo", TitleOrValueTypeValue{42}}}}
+	fooEntries = append(fooEntries, kv{k, vdl.ValueOf(f)})
+	if err := fooTable.Put(ctx, k, f); err != nil {
+		t.Fatalf("fooTable.Put() failed: %v", err)
+	}
+}
+
+type execSelectTest struct {
+	query   string
+	headers []string
+	r       [][]*vdl.Value
+}
+
+type preExecFunctionTest struct {
+	query   string
+	headers []string
+}
+
+type execSelectErrorTest struct {
+	query string
+	err   error
+}
+
+func TestQueryExec(t *testing.T) {
+	setup(t)
+	defer cleanup()
+	basic := []execSelectTest{
+		{
+			// Select values for all customer records.
+			"select v from Customer where t = \"Customer\"",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+				[]*vdl.Value{customerEntries[4].value},
+			},
+		},
+		{
+			// Select values where v.InvoiceNum is nil
+			// Since InvoiceNum does not exist for Invoice,
+			// this will return just customers.
+			"select v from Customer where v.InvoiceNum is nil",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+				[]*vdl.Value{customerEntries[4].value},
+			},
+		},
+		{
+			// Select values where v.InvoiceNum is nil
+			// or v.Name is nil This will select all customers
+			// with the former and all invoices with the latter.
+			// Hence, all records are returned.
+			"select v from Customer where v.InvoiceNum is nil or v.Name is nil",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+				[]*vdl.Value{customerEntries[1].value},
+				[]*vdl.Value{customerEntries[2].value},
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[4].value},
+				[]*vdl.Value{customerEntries[5].value},
+				[]*vdl.Value{customerEntries[6].value},
+				[]*vdl.Value{customerEntries[7].value},
+				[]*vdl.Value{customerEntries[8].value},
+			},
+		},
+		{
+			// Select values where v.InvoiceNum is nil
+			// and v.Name is nil.  Expect nothing returned.
+			"select v from Customer where v.InvoiceNum is nil and v.Name is nil",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Select values where v.InvoiceNum is not nil
+			// This will return just invoices.
+			"select v from Customer where v.InvoiceNum is not nil",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[1].value},
+				[]*vdl.Value{customerEntries[2].value},
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[5].value},
+				[]*vdl.Value{customerEntries[6].value},
+				[]*vdl.Value{customerEntries[7].value},
+				[]*vdl.Value{customerEntries[8].value},
+			},
+		},
+		{
+			// Select values where v.InvoiceNum is not nil
+			// or v.Name is not nil. All records are returned.
+			"select v from Customer where v.InvoiceNum is not nil or v.Name is not nil",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+				[]*vdl.Value{customerEntries[1].value},
+				[]*vdl.Value{customerEntries[2].value},
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[4].value},
+				[]*vdl.Value{customerEntries[5].value},
+				[]*vdl.Value{customerEntries[6].value},
+				[]*vdl.Value{customerEntries[7].value},
+				[]*vdl.Value{customerEntries[8].value},
+			},
+		},
+		{
+			// Select values where v.InvoiceNum is nil and v.Name is not nil.
+			// All customers are returned.
+			"select v from Customer where v.InvoiceNum is nil and v.Name is not nil",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+				[]*vdl.Value{customerEntries[4].value},
+			},
+		},
+		{
+			// Select values where v.InvoiceNum is not nil
+			// and v.Name is not nil.  Expect nothing returned.
+			"select v from Customer where v.InvoiceNum is not nil and v.Name is not nil",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Select keys & values for all customer records.
+			"select k, v from Customer where t = \"Customer\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), customerEntries[0].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), customerEntries[4].value},
+			},
+		},
+		{
+			// Select keys & names for all customer records.
+			"select k, v.Name from Customer where t = \"Customer\"",
+			[]string{"k", "v.Name"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), vdl.ValueOf("John Smith")},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), vdl.ValueOf("Bat Masterson")},
+			},
+		},
+		{
+			// Select both customer and invoice records.
+			// Customer records have Id.
+			// Invoice records have CustId.
+			"select v.Id, v.CustId from Customer",
+			[]string{"v.Id", "v.CustId"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(int64(1)), vdl.ValueOf(nil)},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(1))},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(1))},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(1))},
+				[]*vdl.Value{vdl.ValueOf(int64(2)), vdl.ValueOf(nil)},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(2))},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(2))},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(2))},
+				[]*vdl.Value{vdl.ValueOf(nil), vdl.ValueOf(int64(2))},
+			},
+		},
+		{
+			// Select keys & values fo all invoice records.
+			"select k, v from Customer where t = \"Invoice\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key), customerEntries[1].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key), customerEntries[2].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), customerEntries[3].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key), customerEntries[5].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key), customerEntries[6].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key), customerEntries[7].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), customerEntries[8].value},
+			},
+		},
+		{
+			// Select key, cust id, invoice number and amount for $88 invoices.
+			"select k, v.CustId as ID, v.InvoiceNum as InvoiceNumber, v.Amount as Amt from Customer where t = \"Invoice\" and v.Amount = 88",
+			[]string{"k", "ID", "InvoiceNumber", "Amt"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), vdl.ValueOf(int64(1)), vdl.ValueOf(int64(1005)), vdl.ValueOf(int64(88))},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), vdl.ValueOf(int64(2)), vdl.ValueOf(int64(1006)), vdl.ValueOf(int64(88))},
+			},
+		},
+		{
+			// Select keys & values for all records with a key prefix of "001".
+			"select k, v from Customer where k like \"001%\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), customerEntries[0].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key), customerEntries[1].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key), customerEntries[2].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), customerEntries[3].value},
+			},
+		},
+		{
+			// Select keys & values for all records with a key prefix of "001".
+			"select k, v from Customer where k like \"002%\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), customerEntries[4].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key), customerEntries[5].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key), customerEntries[6].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key), customerEntries[7].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), customerEntries[8].value},
+			},
+		},
+		{
+			// Select keys & values for all records with a key prefix of "001".
+			// or a key prefix of "002".
+			"select k, v from Customer where k like \"001%\" or k like \"002%\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), customerEntries[0].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key), customerEntries[1].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key), customerEntries[2].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), customerEntries[3].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), customerEntries[4].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key), customerEntries[5].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key), customerEntries[6].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key), customerEntries[7].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), customerEntries[8].value},
+			},
+		},
+		{
+			// Select keys & values for all records with a key prefix of "001".
+			// or a key prefix of "002".
+			"select k, v from Customer where k like \"002%\" or k like \"001%\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), customerEntries[0].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key), customerEntries[1].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key), customerEntries[2].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), customerEntries[3].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), customerEntries[4].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key), customerEntries[5].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key), customerEntries[6].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key), customerEntries[7].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), customerEntries[8].value},
+			},
+		},
+		{
+			// Let's play with whitespace and mixed case.
+			"   sElEcT  k,  v from \n  Customer WhErE k lIkE \"002%\" oR k LiKe \"001%\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), customerEntries[0].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key), customerEntries[1].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key), customerEntries[2].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), customerEntries[3].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), customerEntries[4].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key), customerEntries[5].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key), customerEntries[6].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key), customerEntries[7].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), customerEntries[8].value},
+			},
+		},
+		{
+			// Add in a like clause that accepts all strings.
+			"   sElEcT  k,  v from \n  Customer WhErE k lIkE \"002%\" oR k LiKe \"001%\" or k lIkE \"%\"",
+			[]string{"k", "v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[0].key), customerEntries[0].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key), customerEntries[1].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key), customerEntries[2].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key), customerEntries[3].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[4].key), customerEntries[4].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key), customerEntries[5].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key), customerEntries[6].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key), customerEntries[7].value},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key), customerEntries[8].value},
+			},
+		},
+		{
+			// Select id, name for customers whose last name is Masterson.
+			"select v.Id as ID, v.Name as Name from Customer where t = \"Customer\" and v.Name like \"%Masterson\"",
+			[]string{"ID", "Name"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(int64(2)), vdl.ValueOf("Bat Masterson")},
+			},
+		},
+		{
+			// Select records where v.Address.City is "Collins" or type is Invoice.
+			"select v from Customer where v.Address.City = \"Collins\" or t = \"Invoice\"",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[1].value},
+				[]*vdl.Value{customerEntries[2].value},
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[4].value},
+				[]*vdl.Value{customerEntries[5].value},
+				[]*vdl.Value{customerEntries[6].value},
+				[]*vdl.Value{customerEntries[7].value},
+				[]*vdl.Value{customerEntries[8].value},
+			},
+		},
+		{
+			// Select records where v.Address.City is "Collins" and v.InvoiceNum is not nil.
+			"select v from Customer where v.Address.City = \"Collins\" and v.InvoiceNum is not nil",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Select records where v.Address.City is "Collins" and v.InvoiceNum is nil.
+			"select v from Customer where v.Address.City = \"Collins\" and v.InvoiceNum is nil",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[4].value},
+			},
+		},
+		{
+			// Select customer name for customer Id (i.e., key) "001".
+			"select v.Name as Name from Customer where t = \"Customer\" and k = \"001\"",
+			[]string{"Name"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("John Smith")},
+			},
+		},
+		{
+			// Select v where v.Credit.Report.EquifaxReport.Rating = 'A'
+			"select v from Customer where v.Credit.Report.EquifaxReport.Rating = 'A'",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+			},
+		},
+		{
+			// Select v where v.AgencyRating = "Bad"
+			"select v from Customer where v.Credit.Report.EquifaxReport.Rating < 'A' or v.Credit.Report.ExperianReport.Rating = \"Bad\" or v.Credit.Report.TransUnionReport.Rating < 90",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[4].value},
+			},
+		},
+		{
+			// Select records where v.Bar.Baz.Name = "FooBarBaz"
+			"select v from Foo where v.Bar.Baz.Name = \"FooBarBaz\"",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{fooEntries[0].value},
+			},
+		},
+		{
+			// Select records where v.Bar.Baz.TitleOrValue.Value = 42
+			"select v from Foo where v.Bar.Baz.TitleOrValue.Value = 42",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{fooEntries[1].value},
+			},
+		},
+		{
+			// Select records where v.Bar.Baz.TitleOrValue.Title = "Vice President"
+			"select v from Foo where v.Bar.Baz.TitleOrValue.Title = \"Vice President\"",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{fooEntries[0].value},
+			},
+		},
+		{
+			// Select records where v.Address.City = "Collins" or type is Invoice.
+			// Limit 3
+			"select v from Customer where v.Address.City = \"Collins\" or t = \"Invoice\" limit 3",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[1].value},
+				[]*vdl.Value{customerEntries[2].value},
+				[]*vdl.Value{customerEntries[3].value},
+			},
+		},
+		{
+			// Select records where v.Address.City = "Collins" or type is Invoice.
+			// Offset 5
+			"select v from Customer where v.Address.City = \"Collins\" or t = \"Invoice\" offset 5",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[6].value},
+				[]*vdl.Value{customerEntries[7].value},
+				[]*vdl.Value{customerEntries[8].value},
+			},
+		},
+		{
+			// Select records where v.Address.City = "Collins" is "Mountain View".
+			"select v from Customer where v.Address.City = \"Mountain View\"",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Select records where v.Address.City = "Collins" or type is Invoice.
+			// Offset 8
+			"select v from Customer where v.Address.City = \"Collins\" or t = \"Invoice\" offset 8",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Select records where v.Address.City = "Collins" or type is Invoice.
+			// Offset 23
+			"select v from Customer where v.Address.City = \"Collins\" or t = \"Invoice\" offset 23",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Select records where v.Address.City = "Collins" is 84 or type is Invoice.
+			// Limit 3 Offset 2
+			"select v from Customer where v.Address.City = \"Collins\" or t = \"Invoice\" limit 3 offset 2",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[4].value},
+				[]*vdl.Value{customerEntries[5].value},
+			},
+		},
+		{
+			// Select records where v.Address.City = "Collins" or (type is Invoice and v.InvoiceNum is not nil).
+			// Limit 3 Offset 2
+			"select v from Customer where v.Address.City = \"Collins\" or (t = \"Invoice\" and v.InvoiceNum is not nil) limit 3 offset 2",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[4].value},
+				[]*vdl.Value{customerEntries[5].value},
+			},
+		},
+		{
+			// Select records where v.Address.City = "Collins" or (type is Invoice and v.InvoiceNum is nil).
+			// Limit 3 Offset 2
+			"select v from Customer where v.Address.City = \"Collins\" or (t = \"Invoice\" and v.InvoiceNum is nil) limit 3 offset 2",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		// Test functions.
+		{
+			// Select invoice records where date is 2015-03-17
+			"select v from Customer where t = \"Invoice\" and YMD(v.InvoiceDate, \"America/Los_Angeles\") = Date(\"2015-03-17 PDT\")",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[5].value},
+				[]*vdl.Value{customerEntries[6].value},
+			},
+		},
+		{
+			// Now will always be > 2012, so all customer records will be returned.
+			"select v from Customer where Now() > Date(\"2012-03-17 PDT\")",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{customerEntries[0].value},
+				[]*vdl.Value{customerEntries[1].value},
+				[]*vdl.Value{customerEntries[2].value},
+				[]*vdl.Value{customerEntries[3].value},
+				[]*vdl.Value{customerEntries[4].value},
+				[]*vdl.Value{customerEntries[5].value},
+				[]*vdl.Value{customerEntries[6].value},
+				[]*vdl.Value{customerEntries[7].value},
+				[]*vdl.Value{customerEntries[8].value},
+			},
+		},
+		{
+			// Select April 2015 PT invoices.
+			// Note: this wouldn't work for March as daylight saving occurs March 8
+			// and causes comparisons for those days to be off 1 hour.
+			// It would work to use UTC -- see next test.
+			"select k from Customer where YM(v.InvoiceDate, \"America/Los_Angeles\") = YM(Date(\"2015-04-01 PDT\"), \"America/Los_Angeles\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key)},
+			},
+		},
+		{
+			// Select March 2015 UTC invoices.
+			"select k from Customer where YM(v.InvoiceDate, \"UTC\") = YM(Date(\"2015-03-01 UTC\"), \"UTC\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key)},
+			},
+		},
+		{
+			// Select 2015 UTC invoices.
+			"select k from Customer where Y(v.InvoiceDate, \"UTC\") = Y(Date(\"2015-01-01 UTC\"), \"UTC\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[1].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[2].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[3].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key)},
+			},
+		},
+		{
+			// Select the Mar 17 2015 11:14:04 America/Los_Angeles invoice.
+			"select k from Customer where v.InvoiceDate = DateTime(\"2015-03-17 11:14:04 PDT\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+			},
+		},
+		{
+			// Select invoices in the minute Mar 17 2015 11:14 America/Los_Angeles invoice.
+			"select k from Customer where YMDHM(v.InvoiceDate, \"America/Los_Angeles\") = YMDHM(DateTime(\"2015-03-17 11:14:00 PDT\"), \"America/Los_Angeles\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+			},
+		},
+		{
+			// Select invoices in the hour Mar 17 2015 11 hundred America/Los_Angeles invoice.
+			"select k from Customer where YMDH(v.InvoiceDate, \"America/Los_Angeles\") = YMDH(DateTime(\"2015-03-17 11:00:00 PDT\"), \"America/Los_Angeles\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+			},
+		},
+		{
+			// Select invoices on the day Mar 17 2015 America/Los_Angeles invoice.
+			"select k from Customer where YMD(v.InvoiceDate, \"America/Los_Angeles\") = YMD(Date(\"2015-03-17 PDT\"), \"America/Los_Angeles\")",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key)},
+			},
+		},
+		// Test string functions in where clause.
+		{
+			// Select invoices shipped to Any street -- using LowerCase.
+			"select k from Customer where t = \"Invoice\" and LowerCase(v.ShipTo.Street) like \"%any%\"",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key)},
+			},
+		},
+		{
+			// Select invoices shipped to Any street -- using UpperCase.
+			"select k from Customer where t = \"Invoice\" and UpperCase(v.ShipTo.Street) like \"%ANY%\"",
+			[]string{"k"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(customerEntries[5].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[6].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[7].key)},
+				[]*vdl.Value{vdl.ValueOf(customerEntries[8].key)},
+			},
+		},
+		// Select clause functions.
+		// Date function
+		{
+			"select Date(\"2015-07-01 PDT\") from Customer",
+			[]string{"Date"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01)},
+			},
+		},
+		// DateTime function
+		{
+			"select DateTime(\"2015-07-01 01:23:45 PDT\") from Customer",
+			[]string{"DateTime"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+				[]*vdl.Value{vdl.ValueOf(t2015_07_01_01_23_45)},
+			},
+		},
+		// LowerCase function
+		{
+			"select LowerCase(v.Name) as name from Customer where t = \"Customer\"",
+			[]string{"name"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("john smith")},
+				[]*vdl.Value{vdl.ValueOf("bat masterson")},
+			},
+		},
+		// UpperCase function
+		{
+			"select UpperCase(v.Name) as NAME from Customer where t = \"Customer\"",
+			[]string{"NAME"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("JOHN SMITH")},
+				[]*vdl.Value{vdl.ValueOf("BAT MASTERSON")},
+			},
+		},
+		// YMDHMS function
+		{
+			"select k, YMDHMS(v.InvoiceDate, \"America/Los_Angeles\") from Customer where t = \"Invoice\" and k = \"002003\"",
+			[]string{
+				"k",
+				"YMDHMS",
+			},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("002003"), vdl.ValueOf(t2015_04_12_22_16_06)},
+			},
+		},
+		// YMDHM function
+		{
+			"select k, YMDHM(v.InvoiceDate, \"America/Los_Angeles\") from Customer where t = \"Invoice\" and k = \"002003\"",
+			[]string{
+				"k",
+				"YMDHM",
+			},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("002003"), vdl.ValueOf(t2015_04_12_22_16)},
+			},
+		},
+		// YMDH function
+		{
+			"select k, YMDH(v.InvoiceDate, \"America/Los_Angeles\") from Customer where t = \"Invoice\" and k = \"002003\"",
+			[]string{
+				"k",
+				"YMDH",
+			},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("002003"), vdl.ValueOf(t2015_04_12_22)},
+			},
+		},
+		// YMD function
+		{
+			"select k, YMD(v.InvoiceDate, \"America/Los_Angeles\") from Customer where t = \"Invoice\" and k = \"002003\"",
+			[]string{
+				"k",
+				"YMD",
+			},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("002003"), vdl.ValueOf(t2015_04_12)},
+			},
+		},
+		// YM function
+		{
+			"select k, YM(v.InvoiceDate, \"America/Los_Angeles\") from Customer where t = \"Invoice\" and k = \"002003\"",
+			[]string{
+				"k",
+				"YM",
+			},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("002003"), vdl.ValueOf(t2015_04)},
+			},
+		},
+		// Y function
+		{
+			"select k, Y(v.InvoiceDate, \"America/Los_Angeles\") from Customer where t = \"Invoice\" and k = \"001001\"",
+			[]string{
+				"k",
+				"Y",
+			},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf("001001"), vdl.ValueOf(t2015)},
+			},
+		},
+		// Nested functions
+		{
+			"select Y(YM(YMD(YMDH(YMDHM(YMDHMS(v.InvoiceDate, \"America/Los_Angeles\"), \"America/Los_Angeles\"), \"America/Los_Angeles\"), \"America/Los_Angeles\"), \"America/Los_Angeles\"), \"America/Los_Angeles\")  from Customer where t = \"Invoice\" and k = \"001001\"",
+			[]string{"Y"},
+			[][]*vdl.Value{
+				[]*vdl.Value{vdl.ValueOf(t2015)},
+			},
+		},
+		// Bad arg to function.  Expression is false.
+		{
+			"select v from Customer where t = \"Invoice\" and YMD(v.InvoiceDate, v.Foo) = v.InvoiceDate",
+			[]string{"v"},
+			[][]*vdl.Value{},
+		},
+		{
+			// Test that all numeric types can compare to an uint64
+			"select v from Numbers where v.Ui64 = v.B and v.Ui64 = v.Ui16 and v.Ui64 = v.Ui32 and v.Ui64 = v.F64 and v.Ui64 = v.I16 and v.Ui64 = v.I32 and v.Ui64 = v.I64 and v.Ui64 = v.F32 and v.Ui64 = v.C64 and v.Ui64 = v.C128",
+			[]string{"v"},
+			[][]*vdl.Value{
+				[]*vdl.Value{numbersEntries[2].value},
+			},
+		},
+	}
+
+	for _, test := range basic {
+		headers, rs, err := db.Exec(ctx, test.query)
+		if err != nil {
+			t.Errorf("query: %s; got %v, want nil", test.query, err)
+		} else {
+			// Collect results.
+			r := [][]*vdl.Value{}
+			for rs.Advance() {
+				r = append(r, rs.Result())
+			}
+			if !reflect.DeepEqual(test.r, r) {
+				t.Errorf("query: %s; got %v, want %v", test.query, r, test.r)
+			}
+			if !reflect.DeepEqual(test.headers, headers) {
+				t.Errorf("query: %s; got %v, want %v", test.query, headers, test.headers)
+			}
+		}
+	}
+}
+
+// Use Now to verify it is "pre" executed such that all the rows
+// have the same time.
+func TestPreExecFunctions(t *testing.T) {
+	setup(t)
+	defer cleanup()
+	basic := []preExecFunctionTest{
+		{
+			"select Now() from Customer",
+			[]string{
+				"Now",
+			},
+		},
+	}
+
+	for _, test := range basic {
+		headers, rs, err := db.Exec(ctx, test.query)
+		if err != nil {
+			t.Errorf("query: %s; got %v, want nil", test.query, err)
+		} else {
+			// Check that all results are identical.
+			// Collect results.
+			var last []*vdl.Value
+			for rs.Advance() {
+				result := rs.Result()
+				if last != nil && !reflect.DeepEqual(last, result) {
+					t.Errorf("query: %s; got %v, want %v", test.query, result, last)
+				}
+				last = result
+			}
+			if !reflect.DeepEqual(test.headers, headers) {
+				t.Errorf("query: %s; got %v, want %v", test.query, headers, test.headers)
+			}
+		}
+	}
+}
+
+// TODO(jkline): More negative tests here (even though they are tested elsewhere)?
+func TestExecErrors(t *testing.T) {
+	setup(t)
+	defer cleanup()
+	basic := []execSelectErrorTest{
+		{
+			"select a from Customer",
+			syncql.NewErrInvalidSelectField(ctx, 7),
+		},
+		{
+			"select v from Unknown",
+			// The following error text is dependent on the implementation of the query_db.Database interface.
+			syncql.NewErrTableCantAccess(ctx, 14, "Unknown", errors.New("exec_test.test:\"a/db\".Exec: Does not exist: Unknown")),
+		},
+		{
+			"select v from Customer offset -1",
+			// The following error text is dependent on implementation of Database.
+			syncql.NewErrExpected(ctx, 30, "positive integer literal"),
+		},
+	}
+
+	for _, test := range basic {
+		_, rs, err := db.Exec(ctx, test.query)
+		if err == nil {
+			err = rs.Err()
+		}
+		// Test both that the IDs compare and the text compares (since the offset needs to match).
+		// Note: This is a little tricky because the actual error message will contain the calling
+		//       module.
+		wantPrefix := test.err.Error()[:strings.Index(test.err.Error(), ":")]
+		wantSuffix := test.err.Error()[len(wantPrefix)+1:]
+		if verror.ErrorID(err) != verror.ErrorID(test.err) || !strings.HasPrefix(err.Error(), wantPrefix) || !strings.HasSuffix(err.Error(), wantSuffix) {
+			t.Errorf("query: %s; got %v, want %v", test.query, err, test.err)
+		}
+	}
+}
diff --git a/v23/syncbase/nosql/internal/query/demo/db/db.go b/v23/syncbase/nosql/internal/query/demo/db/db.go
index 59a8a9d..9708755 100644
--- a/v23/syncbase/nosql/internal/query/demo/db/db.go
+++ b/v23/syncbase/nosql/internal/query/demo/db/db.go
@@ -9,7 +9,7 @@
 	"fmt"
 	"strings"
 
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/v23"
 	"v.io/v23/context"
 	"v.io/v23/vdl"
diff --git a/v23/syncbase/nosql/internal/query/demo/demo.go b/v23/syncbase/nosql/internal/query/demo/demo.go
index dd46701..b19e0cc 100644
--- a/v23/syncbase/nosql/internal/query/demo/demo.go
+++ b/v23/syncbase/nosql/internal/query/demo/demo.go
@@ -18,7 +18,7 @@
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/demo/db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/demo/reader"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/demo/writer"
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	_ "v.io/x/ref/runtime/factories/generic"
 )
 
diff --git a/v23/syncbase/nosql/internal/query/eval.go b/v23/syncbase/nosql/internal/query/eval.go
index e3ef7f3..6aa7b0b 100644
--- a/v23/syncbase/nosql/internal/query/eval.go
+++ b/v23/syncbase/nosql/internal/query/eval.go
@@ -11,9 +11,9 @@
 
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/conversions"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_checker"
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_functions"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23/vdl"
 )
@@ -449,6 +449,20 @@
 	return &newOp
 }
 
+// Auto-dereference Any and Optional values
+func autoDereference(o *vdl.Value) *vdl.Value {
+	for o.Kind() == vdl.Any || o.Kind() == vdl.Optional {
+		o = o.Elem()
+		if o == nil {
+			break
+		}
+	}
+	if o == nil {
+		o = vdl.ValueOf(nil)
+	}
+	return o
+}
+
 // Resolve a field.  In the special case where a type is evaluated, in addition
 // to a string being returned, and alternate string is returned.  In this case,
 // <string-value>, true, <alt-string> is returned.  In all other cases,
@@ -457,6 +471,8 @@
 	if query_checker.IsKeyField(f) {
 		return vdl.StringValue(k), false, ""
 	}
+	// Auto-dereference Any and Optional values
+	v = autoDereference(v)
 	t := v.Type()
 	if query_checker.IsTypeField(f) {
 		// Types evaluate to two strings, Str and AltStr.
@@ -469,6 +485,8 @@
 	segments := f.Segments
 	// The first segment will always be v (itself), skip it.
 	for i := 1; i < len(segments); i++ {
+		// Auto-dereference Any and Optional values
+		object = autoDereference(object)
 		// object must be a struct in order to look for the next segment.
 		if object.Kind() == vdl.Struct {
 			if object = object.StructFieldByName(segments[i].Value); object == nil {
diff --git a/v23/syncbase/nosql/internal/query/query.go b/v23/syncbase/nosql/internal/query/query.go
index a6cd34c..da87502 100644
--- a/v23/syncbase/nosql/internal/query/query.go
+++ b/v23/syncbase/nosql/internal/query/query.go
@@ -8,9 +8,9 @@
 	"reflect"
 
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_checker"
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_functions"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23/vdl"
 )
diff --git a/v23/syncbase/nosql/internal/query/query_checker/query_checker.go b/v23/syncbase/nosql/internal/query/query_checker/query_checker.go
index ab892a7..ce807c0 100644
--- a/v23/syncbase/nosql/internal/query/query_checker/query_checker.go
+++ b/v23/syncbase/nosql/internal/query/query_checker/query_checker.go
@@ -10,9 +10,9 @@
 	"sort"
 	"strings"
 
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_functions"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 )
 
diff --git a/v23/syncbase/nosql/internal/query/query_checker/query_checker_test.go b/v23/syncbase/nosql/internal/query/query_checker/query_checker_test.go
index 403c2e8..16dd2d7 100644
--- a/v23/syncbase/nosql/internal/query/query_checker/query_checker_test.go
+++ b/v23/syncbase/nosql/internal/query/query_checker/query_checker_test.go
@@ -11,8 +11,8 @@
 	"testing"
 
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_checker"
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23"
 	"v.io/v23/context"
diff --git a/v23/syncbase/nosql/internal/query/query_functions/query_functions.go b/v23/syncbase/nosql/internal/query/query_functions/query_functions.go
index ca83efb..9fb73db 100644
--- a/v23/syncbase/nosql/internal/query/query_functions/query_functions.go
+++ b/v23/syncbase/nosql/internal/query/query_functions/query_functions.go
@@ -8,8 +8,8 @@
 	"strings"
 
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/conversions"
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23/vdl"
 )
diff --git a/v23/syncbase/nosql/internal/query/query_functions/query_functions_test.go b/v23/syncbase/nosql/internal/query/query_functions/query_functions_test.go
index 56c6f39..eef9604 100644
--- a/v23/syncbase/nosql/internal/query/query_functions/query_functions_test.go
+++ b/v23/syncbase/nosql/internal/query/query_functions/query_functions_test.go
@@ -10,9 +10,9 @@
 	"testing"
 	"time"
 
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_functions"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/v23"
 	"v.io/v23/context"
 	_ "v.io/x/ref/runtime/factories/generic"
diff --git a/v23/syncbase/nosql/internal/query/query_parser/query_parser.go b/v23/syncbase/nosql/internal/query/query_parser/query_parser.go
index 08600a7..78bd542 100644
--- a/v23/syncbase/nosql/internal/query/query_parser/query_parser.go
+++ b/v23/syncbase/nosql/internal/query/query_parser/query_parser.go
@@ -14,7 +14,7 @@
 	"time"
 	"unicode/utf8"
 
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23/vdl"
 )
diff --git a/v23/syncbase/nosql/internal/query/query_parser/query_parser_test.go b/v23/syncbase/nosql/internal/query/query_parser/query_parser_test.go
index a1e8961..b118e4d 100644
--- a/v23/syncbase/nosql/internal/query/query_parser/query_parser_test.go
+++ b/v23/syncbase/nosql/internal/query/query_parser/query_parser_test.go
@@ -8,8 +8,8 @@
 	"reflect"
 	"testing"
 
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23"
 	"v.io/v23/context"
diff --git a/v23/syncbase/nosql/internal/query/test/query_test.go b/v23/syncbase/nosql/internal/query/test/query_test.go
index c9db046..86e395e 100644
--- a/v23/syncbase/nosql/internal/query/test/query_test.go
+++ b/v23/syncbase/nosql/internal/query/test/query_test.go
@@ -14,8 +14,8 @@
 
 	"v.io/syncbase/v23/syncbase/nosql/internal/query"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_checker"
-	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/internal/query/query_parser"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
 	"v.io/syncbase/v23/syncbase/nosql/syncql"
 	"v.io/v23"
 	"v.io/v23/context"
@@ -992,6 +992,12 @@
 			nil,
 		},
 		{
+			// Need all keys (single prefix of "").
+			"   sElEcT  k,  v from \n  Customer WhErE k lIkE \"002%\" oR k LiKe \"001%\" or k lIkE \"%\"",
+			[]string{""},
+			nil,
+		},
+		{
 			// All selected rows will have key prefix of "abc".
 			"select k, v from Customer where t = \"Foo.Bar\" and k like \"abc%\"",
 			[]string{"abc"},
diff --git a/v23/syncbase/nosql/model.go b/v23/syncbase/nosql/model.go
index 5bcc64d..1fa5f87 100644
--- a/v23/syncbase/nosql/model.go
+++ b/v23/syncbase/nosql/model.go
@@ -10,6 +10,7 @@
 	"v.io/syncbase/v23/syncbase/util"
 	"v.io/v23/context"
 	"v.io/v23/security/access"
+	"v.io/v23/vdl"
 )
 
 // TODO(sadovsky): Document the access control policy for every method where
@@ -55,6 +56,14 @@
 	// DeleteTable deletes the specified Table.
 	DeleteTable(ctx *context.T, relativeName string) error
 
+	// Exec executes a syncQL query.
+	// If no error is returned, Exec returns an array of headers (i.e., column names)
+	// and a result stream which contains an array of values for each row which matches
+	// the query.  The number of values returned in each row of the result stream will
+	// match the size of the headers string array.
+	// TODO(jkline): Move this to DatabaseHandle to make it work in batches.
+	Exec(ctx *context.T, query string) ([]string, ResultStream, error)
+
 	// BeginBatch creates a new batch. Instead of calling this function directly,
 	// clients are recommended to use the RunInBatch() helper function, which
 	// detects "concurrent batch" errors and handles retries internally.
@@ -229,6 +238,32 @@
 	Cancel()
 }
 
+// ResultStream is an interface for iterating through rows resulting from an Exec.
+type ResultStream interface {
+	// Advance stages an result so the client can retrieve it with Result.
+	// Advance returns true iff there is a result to retrieve. The client must
+	// call Advance before calling Result. The client must call Cancel if it
+	// does not iterate through all results (i.e. until Advance returns false).
+	// Advance may block if a result is not immediately available.
+	Advance() bool
+
+	// Result returns the result that was staged by Advance.
+	// Result may panic if Advance returned false or was not called at all.
+	// Result does not block.
+	Result() []*vdl.Value
+
+	// Err returns a non-nil error iff the stream encountered any errors. Err does
+	// not block.
+	Err() error
+
+	// Cancel notifies the stream provider that it can stop producing results.
+	// The client must call Cancel if it does not iterate through all results
+	// (i.e. until Advance returns false). Cancel is idempotent and can be called
+	// concurrently with a goroutine that is iterating via Advance/Result.
+	// Cancel causes Advance to subsequently return false. Cancel does not block.
+	Cancel()
+}
+
 // SyncGroup is the interface for a SyncGroup in the store.
 type SyncGroup interface {
 	// Create creates a new SyncGroup with the given spec.
diff --git a/v23/syncbase/nosql/internal/query/query_db/doc.go b/v23/syncbase/nosql/query_db/doc.go
similarity index 75%
rename from v23/syncbase/nosql/internal/query/query_db/doc.go
rename to v23/syncbase/nosql/query_db/doc.go
index 2c5b2ed..2cd7e47 100644
--- a/v23/syncbase/nosql/internal/query/query_db/doc.go
+++ b/v23/syncbase/nosql/query_db/doc.go
@@ -2,10 +2,12 @@
 // Use of this source code is governed by a BSD-style
 // license that can be found in the LICENSE file.
 
-// Package query_db defines the interfaces a consumer of the query package would need to
+// Package query_db defines the interfaces a consumer of the query package needs to
 // implement.
 //
 // The Database interface is used to get Table interfaces (by name).
 // The Table interface is used to get a KeyValueStream (by key prefixes).
 // The KeyValueStream is used to consume key value pairs that match the prefixes.
+// The Exec function is a public wrapper around Exec in the internal query package.
+// TODO(jkline): Choose a better name than query_db.
 package query_db
diff --git a/v23/syncbase/nosql/internal/query/query_db/query_db.go b/v23/syncbase/nosql/query_db/query_db.go
similarity index 100%
rename from v23/syncbase/nosql/internal/query/query_db/query_db.go
rename to v23/syncbase/nosql/query_db/query_db.go
diff --git a/v23/syncbase/nosql/query_exec/exec.go b/v23/syncbase/nosql/query_exec/exec.go
new file mode 100644
index 0000000..edff51e
--- /dev/null
+++ b/v23/syncbase/nosql/query_exec/exec.go
@@ -0,0 +1,18 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package query_exec
+
+import (
+	"v.io/syncbase/v23/syncbase/nosql"
+	"v.io/syncbase/v23/syncbase/nosql/internal/query"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
+)
+
+// Exec executes a syncQL query and returns all results as specified by
+// in the query's select clause.  Headers (i.e., column names) are returned
+// separately from the result stream.
+func Exec(db query_db.Database, q string) ([]string, nosql.ResultStream, error) {
+	return query.Exec(db, q)
+}
diff --git a/v23/syncbase/nosql/query_stream.go b/v23/syncbase/nosql/query_stream.go
new file mode 100644
index 0000000..ef103c1
--- /dev/null
+++ b/v23/syncbase/nosql/query_stream.go
@@ -0,0 +1,85 @@
+// Copyright 2015 The Vanadium Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in the LICENSE file.
+
+package nosql
+
+import (
+	"sync"
+
+	wire "v.io/syncbase/v23/services/syncbase/nosql"
+	"v.io/v23/context"
+	"v.io/v23/vdl"
+	"v.io/v23/verror"
+)
+
+type resultStream struct {
+	mu sync.Mutex
+	// cancel cancels the RPC resultStream.
+	cancel context.CancelFunc
+	// call is the RPC resultStream object.
+	call wire.DatabaseExecClientCall
+	// curr is the currently staged result, or nil if nothing is staged.
+	curr []*vdl.Value
+	// err is the first error encountered during streaming. It may also be
+	// populated by a call to Cancel.
+	err error
+	// finished records whether we have called call.Finish().
+	finished bool
+}
+
+var _ ResultStream = (*resultStream)(nil)
+
+func newResultStream(cancel context.CancelFunc, call wire.DatabaseExecClientCall) *resultStream {
+	return &resultStream{
+		cancel: cancel,
+		call:   call,
+	}
+}
+
+func (rs *resultStream) Advance() bool {
+	rs.mu.Lock()
+	defer rs.mu.Unlock()
+	if rs.err != nil || rs.finished {
+		return false
+	}
+	if !rs.call.RecvStream().Advance() {
+		if rs.call.RecvStream().Err() != nil {
+			rs.err = rs.call.RecvStream().Err()
+		} else {
+			rs.err = rs.call.Finish()
+			rs.cancel() // TODO(jkline): Copied from stream.go, is this needed?
+			rs.finished = true
+		}
+		return false
+	}
+	curr := rs.call.RecvStream().Value()
+	rs.curr = curr
+	return true
+}
+
+func (rs *resultStream) Result() []*vdl.Value {
+	rs.mu.Lock()
+	defer rs.mu.Unlock()
+	if rs.curr == nil {
+		panic("nothing staged")
+	}
+	return rs.curr
+}
+
+func (rs *resultStream) Err() error {
+	rs.mu.Lock()
+	defer rs.mu.Unlock()
+	if rs.err == nil {
+		return nil
+	}
+	return rs.err
+}
+
+// TODO(jkline): Make Cancel non-blocking (TODO copied from stream.go)
+func (rs *resultStream) Cancel() {
+	rs.mu.Lock()
+	defer rs.mu.Unlock()
+	rs.cancel()
+	rs.err = verror.New(verror.ErrCanceled, nil)
+}
diff --git a/v23/syncbase/testutil/util.go b/v23/syncbase/testutil/util.go
index 405bbf5..0466f3c 100644
--- a/v23/syncbase/testutil/util.go
+++ b/v23/syncbase/testutil/util.go
@@ -22,6 +22,8 @@
 	"v.io/v23/rpc"
 	"v.io/v23/security"
 	"v.io/v23/security/access"
+	"v.io/v23/vdl"
+	"v.io/v23/verror"
 	"v.io/x/lib/vlog"
 	tsecurity "v.io/x/ref/test/testutil"
 )
@@ -127,6 +129,41 @@
 	}
 }
 
+func CheckExec(t *testing.T, ctx *context.T, db nosql.Database, q string, wantHeaders []string, wantResults [][]*vdl.Value) {
+	gotHeaders, it, err := db.Exec(ctx, q)
+	if err != nil {
+		t.Errorf("query %q: got %v, want nil", q, err)
+	}
+	if !reflect.DeepEqual(gotHeaders, wantHeaders) {
+		t.Errorf("query %q: got %v, want %v", q, gotHeaders, wantHeaders)
+	}
+	gotResults := [][]*vdl.Value{}
+	for it.Advance() {
+		gotResult := it.Result()
+		gotResults = append(gotResults, gotResult)
+	}
+	if it.Err() != nil {
+		t.Errorf("query %q: got %v, want nil", q, it.Err())
+	}
+	if !reflect.DeepEqual(gotResults, wantResults) {
+		t.Errorf("query %q: got %v, want %v", q, gotResults, wantResults)
+	}
+}
+
+func CheckExecError(t *testing.T, ctx *context.T, db nosql.Database, q string, wantErrorID verror.ID) {
+	_, rs, err := db.Exec(ctx, q)
+	if err == nil {
+		if rs.Advance() {
+			t.Errorf("query %q: got true, want false", q)
+		}
+		err = rs.Err()
+	}
+	if verror.ErrorID(err) != wantErrorID {
+		t.Errorf("%q", verror.DebugString(err))
+		t.Errorf("query %q: got %v, want: %v", q, verror.ErrorID(err), wantErrorID)
+	}
+}
+
 ////////////////////////////////////////
 // Internal helpers
 
diff --git a/x/ref/services/syncbase/server/nosql/database.go b/x/ref/services/syncbase/server/nosql/database.go
index b9f7d7b..0971923 100644
--- a/x/ref/services/syncbase/server/nosql/database.go
+++ b/x/ref/services/syncbase/server/nosql/database.go
@@ -13,6 +13,9 @@
 	"time"
 
 	wire "v.io/syncbase/v23/services/syncbase/nosql"
+	"v.io/syncbase/v23/syncbase/nosql/query_db"
+	"v.io/syncbase/v23/syncbase/nosql/query_exec"
+	prefixutil "v.io/syncbase/v23/syncbase/util"
 	"v.io/syncbase/x/ref/services/syncbase/server/interfaces"
 	"v.io/syncbase/x/ref/services/syncbase/server/util"
 	"v.io/syncbase/x/ref/services/syncbase/server/watchable"
@@ -20,7 +23,9 @@
 	"v.io/v23/context"
 	"v.io/v23/rpc"
 	"v.io/v23/security/access"
+	"v.io/v23/vdl"
 	"v.io/v23/verror"
+	"v.io/v23/vom"
 	"v.io/x/lib/vlog"
 )
 
@@ -202,6 +207,46 @@
 	return err
 }
 
+func (d *databaseReq) Exec(ctx *context.T, call wire.DatabaseExecServerCall, q string) error {
+	impl := func(headers []string, rs ResultStream, err error) error {
+		if err != nil {
+			return err
+		}
+		sender := call.SendStream()
+		// Push the headers first -- the client will retrieve them and return
+		// them separately from the results.
+		var resultHeaders []*vdl.Value
+		for _, header := range headers {
+			resultHeaders = append(resultHeaders, vdl.ValueOf(header))
+		}
+		sender.Send(resultHeaders)
+		for rs.Advance() {
+			result := rs.Result()
+			sender.Send(result)
+		}
+		return rs.Err()
+	}
+	var st store.StoreReader
+	if d.batchId != nil {
+		st = d.batchReader()
+	} else {
+		sn := d.st.NewSnapshot()
+		st = sn
+		defer sn.Close()
+	}
+	// queryDb implements query_db.Database
+	// which is needed by the query package's
+	// Exec function.
+	db := &queryDb{
+		ctx:  ctx,
+		call: call,
+		req:  d,
+		st:   st,
+	}
+
+	return impl(query_exec.Exec(db, q))
+}
+
 func (d *databaseReq) SetPermissions(ctx *context.T, call rpc.ServerCall, perms access.Permissions, version string) error {
 	if !d.exists {
 		return verror.New(verror.ErrNoExist, ctx, d.name)
@@ -246,6 +291,36 @@
 }
 
 ////////////////////////////////////////
+// ResultStream interface
+
+// ResultStream is an interface for iterating through results (a.k.a, rows) returned from a
+// query.  Each resulting rows are arrays of vdl objects.
+type ResultStream interface {
+	// Advance stages an element so the client can retrieve it with Result.
+	// Advance returns true iff there is a result to retrieve. The client must
+	// call Advance before calling Result. The client must call Cancel if it
+	// does not iterate through all elements (i.e. until Advance returns false).
+	// Advance may block if an element is not immediately available.
+	Advance() bool
+
+	// Result returns the row (i.e., array of vdl Values) that was staged by Advance.
+	// Result may panic if Advance returned false or was not called at all.
+	// Result does not block.
+	Result() []*vdl.Value
+
+	// Err returns a non-nil error iff the stream encountered any errors. Err does
+	// not block.
+	Err() error
+
+	// Cancel notifies the ResultStream provider that it can stop producing results.
+	// The client must call Cancel if it does not iterate through all results
+	// (i.e. until Advance returns false). Cancel is idempotent and can be called
+	// concurrently with a goroutine that is iterating via Advance/Result.
+	// Cancel causes Advance to subsequently return false. Cancel does not block.
+	Cancel()
+}
+
+////////////////////////////////////////
 // interfaces.Database methods
 
 func (d *database) St() store.Store {
@@ -295,6 +370,130 @@
 }
 
 ////////////////////////////////////////
+// query_db implementations
+
+// Implement query_db's Database, Table and KeyValueStream interfaces.
+type queryDb struct {
+	ctx  *context.T
+	call wire.DatabaseExecServerCall
+	req  *databaseReq
+	st   store.StoreReader
+}
+
+func (db *queryDb) GetContext() *context.T {
+	return db.ctx
+}
+
+func (db *queryDb) GetTable(name string) (query_db.Table, error) {
+	tDb := &tableDb{
+		qdb: db,
+		req: &tableReq{
+			name: name,
+			d:    db.req,
+		},
+	}
+	// Now that we have a table, we need to check permissions.
+	if err := util.Get(db.ctx, db.call, db.st, tDb.req, &tableData{}); err != nil {
+		return nil, err
+	}
+	return tDb, nil
+}
+
+type tableDb struct {
+	qdb *queryDb
+	req *tableReq
+}
+
+func (t *tableDb) Scan(prefixes []string) (query_db.KeyValueStream, error) {
+	return &kvs{
+		t:        t,
+		prefixes: prefixes,
+		curr:     -1,
+		validRow: false,
+		it:       nil,
+		err:      nil,
+	}, nil
+}
+
+type kvs struct {
+	t         *tableDb
+	prefixes  []string
+	curr      int // current index into prefixes, -1 at start
+	validRow  bool
+	currKey   string
+	currValue *vdl.Value
+	it        store.Stream // current prefix key value stream
+	err       error
+}
+
+func (s *kvs) Advance() bool {
+	if s.err != nil {
+		return false
+	}
+	if s.curr == -1 {
+		s.curr++
+	}
+	for s.curr < len(s.prefixes) {
+		if s.it == nil {
+			start := prefixutil.PrefixRangeStart(s.prefixes[s.curr])
+			limit := prefixutil.PrefixRangeLimit(s.prefixes[s.curr])
+			s.it = s.t.qdb.st.Scan(util.ScanRangeArgs(util.JoinKeyParts(util.RowPrefix, s.t.req.name), string(start), string(limit)))
+		}
+		if s.it.Advance() {
+			// key
+			keyBytes := s.it.Key(nil)
+			parts := util.SplitKeyParts(string(keyBytes))
+			s.currKey = parts[len(parts)-1]
+			// value
+			valueBytes := s.it.Value(nil)
+			var currValue *vdl.Value
+			if err := vom.Decode(valueBytes, &currValue); err != nil {
+				s.validRow = false
+				s.err = err
+				return false
+			}
+			s.currValue = currValue
+			s.validRow = true
+			return true
+		}
+		// Advance returned false.  It could be an err, or it could
+		// be we've reached the end.
+		if err := s.it.Err(); err != nil {
+			s.validRow = false
+			s.err = err
+			return false
+		}
+		// We've reached the end of the iterator for this prefix.
+		// Jump to the next one.
+		s.curr++
+		s.it = nil
+		s.validRow = false
+	}
+	// There are no more prefixes to scan.
+	return false
+}
+
+func (s *kvs) KeyValue() (string, *vdl.Value) {
+	if !s.validRow {
+		return "", nil
+	}
+	return s.currKey, s.currValue
+}
+
+func (s *kvs) Err() error {
+	return s.err
+}
+
+func (s *kvs) Cancel() {
+	if s.it != nil {
+		s.it.Cancel()
+		s.it = nil
+	}
+	// set curr to end of prefixes so Advance will return false
+	s.curr = len(s.prefixes)
+}
+
+////////////////////////////////////////
 // Internal helpers
 
 func (d *databaseReq) batchReader() store.StoreReader {