Stash list table + (mostly) function procedure

This commit is contained in:
Nathan Gabrielson
2025-06-04 13:38:28 -07:00
parent 6251b3513f
commit 32b529bfc3
14 changed files with 541 additions and 66 deletions

View File

@@ -77,7 +77,7 @@ func (cmd StashClearCmd) Exec(ctx context.Context, commandStr string, args []str
return 1
}
err := dEnv.DoltDB(ctx).RemoveAllStashes(ctx)
err := dEnv.DoltDB(ctx).RemoveAllStashes(ctx, "dolt-cli")
if err != nil {
return commands.HandleVErrAndExitCode(errhand.VerboseErrorFromError(err), usage)
}

View File

@@ -95,12 +95,12 @@ func (cmd StashDropCmd) Exec(ctx context.Context, commandStr string, args []stri
}
func dropStashAtIdx(ctx context.Context, dEnv *env.DoltEnv, idx int) error {
stashHash, err := dEnv.DoltDB(ctx).GetStashHashAtIdx(ctx, idx)
stashHash, err := dEnv.DoltDB(ctx).GetStashHashAtIdx(ctx, idx, "dolt-cli")
if err != nil {
return err
}
err = dEnv.DoltDB(ctx).RemoveStashAtIdx(ctx, idx)
err = dEnv.DoltDB(ctx).RemoveStashAtIdx(ctx, idx, "dolt-cli")
if err != nil {
return err
}

View File

@@ -125,7 +125,7 @@ func (cmd StashPopCmd) Exec(ctx context.Context, commandStr string, args []strin
}
func applyStashAtIdx(ctx *sql.Context, dEnv *env.DoltEnv, curWorkingRoot doltdb.RootValue, idx int) (bool, error) {
stashRoot, headCommit, meta, err := dEnv.DoltDB(ctx).GetStashRootAndHeadCommitAtIdx(ctx, idx)
stashRoot, headCommit, meta, err := dEnv.DoltDB(ctx).GetStashRootAndHeadCommitAtIdx(ctx, idx, "dolt-cli")
if err != nil {
return false, err
}

View File

@@ -242,7 +242,7 @@ func stashChanges(ctx context.Context, dEnv *env.DoltEnv, apr *argparser.ArgPars
return err
}
err = dEnv.DoltDB(ctx).AddStash(ctx, commit, roots.Staged, datas.NewStashMeta(curBranchName, commitMeta.Description, doltdb.FlattenTableNames(addedTblsToStage)))
err = dEnv.DoltDB(ctx).AddStash(ctx, commit, roots.Staged, datas.NewStashMeta(curBranchName, commitMeta.Description, doltdb.FlattenTableNames(addedTblsToStage)), "dolt-cli")
if err != nil {
return err
}

View File

@@ -2076,8 +2076,8 @@ func (ddb *DoltDB) GetBranchesByRootHash(ctx context.Context, rootHash hash.Hash
// AddStash takes current branch head commit, stash root value and stash metadata to create a new stash.
// It stores the new stash object in stash list Dataset, which can be created if it does not exist.
// Otherwise, it updates the stash list Dataset as there can only be one stashes Dataset.
func (ddb *DoltDB) AddStash(ctx context.Context, head *Commit, stash RootValue, meta *datas.StashMeta) error {
stashesDS, err := ddb.db.GetDataset(ctx, ref.NewStashRef().String())
func (ddb *DoltDB) AddStash(ctx context.Context, head *Commit, stash RootValue, meta *datas.StashMeta, stashName string) error {
stashesDS, err := ddb.db.GetDataset(ctx, ref.NewStashRef(stashName).String())
if err != nil {
return err
}
@@ -2159,8 +2159,8 @@ func (ddb *DoltDB) GetStatistics(ctx context.Context) (prolly.Map, error) {
// It removes a Stash message from stash list Dataset, which cannot be performed
// by database Delete function. This function removes a single stash only and stash
// list dataset does not get removed if there are no entries left.
func (ddb *DoltDB) RemoveStashAtIdx(ctx context.Context, idx int) error {
stashesDS, err := ddb.db.GetDataset(ctx, ref.NewStashRef().String())
func (ddb *DoltDB) RemoveStashAtIdx(ctx context.Context, idx int, stashName string) error {
stashesDS, err := ddb.db.GetDataset(ctx, ref.NewStashRef(stashName).String())
if err != nil {
return err
}
@@ -2186,7 +2186,7 @@ func (ddb *DoltDB) RemoveStashAtIdx(ctx context.Context, idx int) error {
}
// if the stash list is empty, remove the stash list Dataset from the database
if stashListCount == 0 {
return ddb.RemoveAllStashes(ctx)
return ddb.RemoveAllStashes(ctx, stashName)
}
stashesDS, err = ddb.db.UpdateStashList(ctx, stashesDS, stashListAddr)
@@ -2195,31 +2195,41 @@ func (ddb *DoltDB) RemoveStashAtIdx(ctx context.Context, idx int) error {
// RemoveAllStashes removes the stash list Dataset from the database,
// which equivalent to removing Stash entries from the stash list.
func (ddb *DoltDB) RemoveAllStashes(ctx context.Context) error {
err := ddb.deleteRef(ctx, ref.NewStashRef(), nil, "")
func (ddb *DoltDB) RemoveAllStashes(ctx context.Context, stashName string) error {
err := ddb.deleteRef(ctx, ref.NewStashRef(stashName), nil, "")
if err == ErrBranchNotFound {
return nil
}
return err
}
var stashRefFilter = map[ref.RefType]struct{}{ref.StashRefType: {}}
// GetStashes returns array of Stash objects containing all stash entries in the stash list Dataset.
func (ddb *DoltDB) GetStashes(ctx context.Context) ([]*Stash, error) {
stashesDS, err := ddb.db.GetDataset(ctx, ref.NewStashRef().String())
stashRefs, err := ddb.GetRefsOfType(ctx, stashRefFilter)
if err != nil {
return nil, err
}
if !stashesDS.HasHead() {
return []*Stash{}, nil
var stashList []*Stash
for _, stash := range stashRefs {
stashDS, err := ddb.db.GetDataset(ctx, ref.NewStashRef(stash.String()).String())
if err != nil {
return nil, err
}
newStashes, err := getStashList(ctx, stashDS, ddb.vrw, ddb.NodeStore())
if err != nil {
return nil, err
}
stashList = append(stashList, newStashes...)
}
return getStashList(ctx, stashesDS, ddb.vrw, ddb.NodeStore())
return stashList, nil
}
// GetStashHashAtIdx returns hash address only of the stash at given index.
func (ddb *DoltDB) GetStashHashAtIdx(ctx context.Context, idx int) (hash.Hash, error) {
ds, err := ddb.db.GetDataset(ctx, ref.NewStashRef().String())
func (ddb *DoltDB) GetStashHashAtIdx(ctx context.Context, idx int, stashName string) (hash.Hash, error) {
ds, err := ddb.db.GetDataset(ctx, ref.NewStashRef(stashName).String())
if err != nil {
return hash.Hash{}, err
}
@@ -2233,8 +2243,8 @@ func (ddb *DoltDB) GetStashHashAtIdx(ctx context.Context, idx int) (hash.Hash, e
// GetStashRootAndHeadCommitAtIdx returns root value of stash working set and head commit of the branch that the stash was made on
// of the stash at given index.
func (ddb *DoltDB) GetStashRootAndHeadCommitAtIdx(ctx context.Context, idx int) (RootValue, *Commit, *datas.StashMeta, error) {
ds, err := ddb.db.GetDataset(ctx, ref.NewStashRef().String())
func (ddb *DoltDB) GetStashRootAndHeadCommitAtIdx(ctx context.Context, idx int, stashName string) (RootValue, *Commit, *datas.StashMeta, error) {
ds, err := ddb.db.GetDataset(ctx, ref.NewStashRef(stashName).String())
if err != nil {
return nil, nil, nil, err
}

View File

@@ -221,6 +221,7 @@ var getGeneratedSystemTables = func() []string {
GetRemotesTableName(),
GetHelpTableName(),
GetBackupsTableName(),
GetStashesTableName(),
}
}
@@ -393,6 +394,10 @@ var GetBackupsTableName = func() string {
return BackupsTableName
}
var GetStashesTableName = func() string {
return StashesTableName
}
const (
// LogTableName is the log system table name
LogTableName = "dolt_log"
@@ -444,6 +449,9 @@ const (
// StatisticsTableName is the statistics system table name
StatisticsTableName = "dolt_statistics"
// StashesTableName is the stashes system table name
StashesTableName = "dolt_stashes"
)
const (

View File

@@ -197,7 +197,7 @@ func Parse(str string) (DoltRef, error) {
str = str[len(prefix):]
switch rType {
case StashRefType:
return NewStashRef(), nil
return NewStashRef(str), nil
default:
panic("unknown type " + rType)
}

View File

@@ -18,9 +18,6 @@ import (
"strings"
)
// StashRefName is a dummy name, and there cannot be more than one stash ref.
const StashRefName = "stashes"
type StashRef struct {
stash string
}
@@ -28,8 +25,7 @@ type StashRef struct {
var _ DoltRef = StashRef{}
// NewStashRef creates a reference to a stashes list. There cannot be more than one stashRef.
func NewStashRef() StashRef {
stashName := StashRefName
func NewStashRef(stashName string) StashRef {
if IsRef(stashName) {
prefix := PrefixForType(StashRefType)
if strings.HasPrefix(stashName, prefix) {

View File

@@ -582,6 +582,14 @@ func (db Database) getTableInsensitive(ctx *sql.Context, head *doltdb.Commit, ds
if !resolve.UseSearchPath || isDoltgresSystemTable {
dt, found = dtables.NewRemotesTable(ctx, db.ddb, lwrName), true
}
case doltdb.StashesTableName, doltdb.GetStashesTableName():
isDoltgresSystemTable, err := resolve.IsDoltgresSystemTable(ctx, tname, root)
if err != nil {
return nil, false, err
}
if !resolve.UseSearchPath || isDoltgresSystemTable {
dt, found = dtables.NewStashesTable(ctx, db.ddb, lwrName), true
}
case doltdb.CommitsTableName, doltdb.GetCommitsTableName():
isDoltgresSystemTable, err := resolve.IsDoltgresSystemTable(ctx, tname, root)
if err != nil {

View File

@@ -81,7 +81,7 @@ func doDoltStash(ctx *sql.Context, args []string) (string, error) {
case "push":
status, err = doStashPush(ctx, dSess, dbData, roots, apr)
case "pop":
status, err = doStashPop(ctx, dbData, apr, roots)
status, err = doStashPop(ctx, dbData, apr)
case "drop":
status, err = doStashDrop(ctx, dbData, apr)
case "clear":
@@ -104,7 +104,7 @@ func doStashPush(ctx *sql.Context, dSess *dsess.DoltSession, dbData env.DbData[*
return "", err
}
if !hasChanges {
return "", fmt.Errorf("no local changes to save") //What should I do here?
return "", fmt.Errorf("no local changes to save")
}
roots, err = actions.StageModifiedAndDeletedTables(ctx, roots)
@@ -155,7 +155,7 @@ func doStashPush(ctx *sql.Context, dSess *dsess.DoltSession, dbData env.DbData[*
return "", err
}
err = dbData.Ddb.AddStash(ctx, commit, roots.Staged, datas.NewStashMeta(curBranchName, commitMeta.Description, doltdb.FlattenTableNames(addedTblsToStage)))
err = dbData.Ddb.AddStash(ctx, commit, roots.Staged, datas.NewStashMeta(curBranchName, commitMeta.Description, doltdb.FlattenTableNames(addedTblsToStage)), "stashes")
if err != nil {
return "", err
}
@@ -180,7 +180,7 @@ func doStashPush(ctx *sql.Context, dSess *dsess.DoltSession, dbData env.DbData[*
return status, nil
}
func doStashPop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argparser.ArgParseResults, roots doltdb.Roots) (string, error) {
func doStashPop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argparser.ArgParseResults) (string, error) {
var idx = 0
var err error
if apr.NArg() == 2 {
@@ -201,7 +201,7 @@ func doStashPop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argparse
}
curWorkingRoot := workingSet.WorkingRoot()
stashRoot, headCommit, meta, err := dbData.Ddb.GetStashRootAndHeadCommitAtIdx(ctx, idx)
stashRoot, headCommit, meta, err := dbData.Ddb.GetStashRootAndHeadCommitAtIdx(ctx, idx, "stashes")
if err != nil {
return "", err
}
@@ -257,39 +257,25 @@ func doStashPop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argparse
return status, nil
}
//err = dEnv.UpdateWorkingRoot(ctx, result.Root)
var h hash.Hash
var wsRef ref.WorkingSetRef
ws, err := env.WorkingSet(ctx, dbData.Ddb, dbData.Rsr)
if err == doltdb.ErrWorkingSetNotFound {
// first time updating root
wsRef, err = ref.WorkingSetRefForHead(headRef)
if err != nil {
return "", err
}
ws = doltdb.EmptyWorkingSet(wsRef).WithWorkingRoot(result.Root).WithStagedRoot(result.Root)
} else if err != nil {
return "", err
} else {
h, err = ws.HashOf()
if err != nil {
return "", err
}
wsRef = ws.Ref()
}
wsm := &datas.WorkingSetMeta{
Timestamp: uint64(time.Now().Unix()),
Description: "updated from dolt environment",
}
err = dbData.Ddb.UpdateWorkingSet(ctx, wsRef, ws.WithWorkingRoot(result.Root), h, wsm, nil)
err = updateWorkingRoot(ctx, dbData, result.Root)
if err != nil {
return "", err
}
headRoot, err := headCommit.GetRootValue(ctx)
if err != nil {
return "", err
}
ws, err := env.WorkingSet(ctx, dbData.Ddb, dbData.Rsr)
if err != nil {
return "", err
}
roots := doltdb.Roots{
Head: headRoot,
Working: ws.WorkingRoot(),
Staged: ws.StagedRoot(),
}
// added tables need to be staged
// since these tables are coming from a stash, don't filter for ignored table names.
roots, err = actions.StageTables(ctx, roots, doltdb.ToTableNames(meta.TablesToStage, doltdb.DefaultSchemaName), false)
@@ -302,12 +288,12 @@ func doStashPop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argparse
return "", err
}
stashHash, err := dbData.Ddb.GetStashHashAtIdx(ctx, idx)
stashHash, err := dbData.Ddb.GetStashHashAtIdx(ctx, idx, "stashes")
if err != nil {
return "", err
}
err = dbData.Ddb.RemoveStashAtIdx(ctx, idx)
err = dbData.Ddb.RemoveStashAtIdx(ctx, idx, "stashes")
if err != nil {
return "", err
}
@@ -327,12 +313,12 @@ func doStashDrop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argpars
}
}
stashHash, err := dbData.Ddb.GetStashHashAtIdx(ctx, idx)
stashHash, err := dbData.Ddb.GetStashHashAtIdx(ctx, idx, "stashes")
if err != nil {
return "", err
}
err = dbData.Ddb.RemoveStashAtIdx(ctx, idx)
err = dbData.Ddb.RemoveStashAtIdx(ctx, idx, "stashes")
if err != nil {
return "", err
}
@@ -343,7 +329,7 @@ func doStashDrop(ctx *sql.Context, dbData env.DbData[*sql.Context], apr *argpars
}
func doStashClear(ctx *sql.Context, dbData env.DbData[*sql.Context]) error {
err := dbData.Ddb.RemoveAllStashes(ctx)
err := dbData.Ddb.RemoveAllStashes(ctx, "stashes")
if err != nil {
return err
}
@@ -528,3 +514,44 @@ func bulkDbEaFactory(dbData env.DbData[*sql.Context]) editor.DbEaFactory {
}
return editor.NewBulkImportTEAFactory(dbData.Ddb.ValueReadWriter(), tmpDir)
}
func updateWorkingRoot(ctx *sql.Context, dbData env.DbData[*sql.Context], newRoot doltdb.RootValue) error {
//err = dEnv.UpdateWorkingRoot(ctx, result.Root)
var h hash.Hash
var wsRef ref.WorkingSetRef
headRef, err := dbData.Rsr.CWBHeadRef(ctx)
if err != nil {
return err
}
ws, err := env.WorkingSet(ctx, dbData.Ddb, dbData.Rsr)
if err == doltdb.ErrWorkingSetNotFound {
// first time updating root
wsRef, err = ref.WorkingSetRefForHead(headRef)
if err != nil {
return err
}
ws = doltdb.EmptyWorkingSet(wsRef).WithWorkingRoot(newRoot).WithStagedRoot(newRoot)
} else if err != nil {
return err
} else {
h, err = ws.HashOf()
if err != nil {
return err
}
wsRef = ws.Ref()
}
wsm := &datas.WorkingSetMeta{
Timestamp: uint64(time.Now().Unix()),
Description: "updated from dolt environment",
}
err = dbData.Ddb.UpdateWorkingSet(ctx, wsRef, ws.WithWorkingRoot(newRoot), h, wsm, nil)
if err != nil {
return err
}
return nil
}

View File

@@ -0,0 +1,210 @@
// Copyright 2025 Dolthub, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package dtables
import (
"fmt"
"github.com/dolthub/dolt/go/libraries/doltcore/doltdb"
"github.com/dolthub/dolt/go/libraries/doltcore/schema"
"github.com/dolthub/dolt/go/libraries/doltcore/sqle/dsess"
"github.com/dolthub/dolt/go/libraries/doltcore/sqle/index"
"github.com/dolthub/go-mysql-server/sql"
"github.com/dolthub/go-mysql-server/sql/types"
"io"
)
const stashesDefaultRowCount = 5
var _ sql.Table = (*StashesTable)(nil)
var _ sql.UpdatableTable = (*StashesTable)(nil)
var _ sql.DeletableTable = (*StashesTable)(nil)
var _ sql.InsertableTable = (*StashesTable)(nil)
var _ sql.ReplaceableTable = (*StashesTable)(nil)
var _ sql.StatisticsTable = (*StashesTable)(nil)
type StashesTable struct {
ddb *doltdb.DoltDB
tableName string
}
func NewStashesTable(_ *sql.Context, ddb *doltdb.DoltDB, tableName string) sql.Table {
return &StashesTable{ddb, tableName}
}
func (st *StashesTable) DataLength(ctx *sql.Context) (uint64, error) {
numBytesPerRow := schema.SchemaAvgLength(st.Schema())
numRows, _, err := st.RowCount(ctx)
if err != nil {
return 0, err
}
return numBytesPerRow * numRows, nil
}
func (st *StashesTable) RowCount(_ *sql.Context) (uint64, bool, error) {
return stashesDefaultRowCount, false, nil
} //Todo: What row count?
// Name is a sql.Table interface function which returns the name of the table
func (st *StashesTable) Name() string {
return st.tableName
}
// String is a sql.Table interface function which returns the name of the table
func (st *StashesTable) String() string {
return st.tableName
}
// Schema is a sql.Table interface function that gets the sql.Schema of the remotes system table
func (st *StashesTable) Schema() sql.Schema {
return []*sql.Column{
{Name: "stash reference", Type: types.Text, Source: st.tableName, PrimaryKey: false, Nullable: false},
{Name: "stash id", Type: types.Text, Source: st.tableName, PrimaryKey: false, Nullable: false},
{Name: "branch", Type: types.Text, Source: st.tableName, PrimaryKey: false, Nullable: false},
{Name: "hash", Type: types.Text, Source: st.tableName, PrimaryKey: false, Nullable: false},
{Name: "commit message", Type: types.Text, Source: st.tableName, PrimaryKey: false, Nullable: true},
}
}
// Collation implements the sql.Table interface.
func (st *StashesTable) Collation() sql.CollationID {
return sql.Collation_Default
}
// Partitions is a sql.Table interface function that returns a partition of the data. Currently the data is unpartitioned.
func (st *StashesTable) Partitions(*sql.Context) (sql.PartitionIter, error) {
return index.SinglePartitionIterFromNomsMap(nil), nil
}
// PartitionRows is a sql.Table interface function that gets a row iterator for a partition
func (st *StashesTable) PartitionRows(ctx *sql.Context, part sql.Partition) (sql.RowIter, error) {
return NewStashItr(ctx, st.ddb)
}
func (st *StashesTable) Updater(ctx *sql.Context) sql.RowUpdater {
return stashWriter{st}
}
func (st *StashesTable) Inserter(*sql.Context) sql.RowInserter {
return stashWriter{st}
}
// Deleter returns a RowDeleter for this table. The RowDeleter will get one call to Delete for each row to be deleted,
// and will end with a call to Close() to finalize the delete operation.
func (st *StashesTable) Deleter(*sql.Context) sql.RowDeleter {
return stashWriter{st}
}
// Replacer returns a RowReplacer for this table. The RowReplacer will have Insert and optionally Delete called once
// for each row, followed by a call to Close() when all rows have been processed.
func (st *StashesTable) Replacer(ctx *sql.Context) sql.RowReplacer {
return stashWriter{st}
}
type StashItr struct {
stashes []*doltdb.Stash
idx int
}
// NewStashItr creates a StashItr from the current environment.
func NewStashItr(ctx *sql.Context, ddb *doltdb.DoltDB) (*StashItr, error) {
dbName := ctx.GetCurrentDatabase()
if len(dbName) == 0 {
return nil, fmt.Errorf("Empty database name.")
}
sess := dsess.DSessFromSess(ctx.Session)
dbData, ok := sess.GetDbData(ctx, dbName)
if !ok {
return nil, sql.ErrDatabaseNotFound.New(dbName)
}
stashes, err := dbData.Ddb.GetStashes(ctx)
if err != nil {
return nil, err
}
return &StashItr{stashes, 0}, nil
}
// Next retrieves the next row. It will return io.EOF if it's the last row.
// After retrieving the last row, Close will be automatically closed.
func (itr *StashItr) Next(ctx *sql.Context) (sql.Row, error) {
if itr.idx >= len(itr.stashes) {
return nil, io.EOF
}
defer func() {
itr.idx++
}()
stash := itr.stashes[itr.idx]
commitHash, err := stash.HeadCommit.HashOf()
if err != nil {
return nil, err
}
return sql.NewRow("stashes/stashes", stash.Name, stash.BranchName, commitHash.String(), stash.Description), nil
}
// Close closes the iterator.
func (itr *StashItr) Close(*sql.Context) error {
return nil
}
var _ sql.RowReplacer = stashWriter{nil}
var _ sql.RowUpdater = stashWriter{nil}
var _ sql.RowInserter = stashWriter{nil}
var _ sql.RowDeleter = stashWriter{nil}
type stashWriter struct {
rt *StashesTable
}
// Insert inserts the row given, returning an error if it cannot. Insert will be called once for each row to process
// for the insert operation, which may involve many rows. After all rows in an operation have been processed, Close
// is called.
func (bWr stashWriter) Insert(ctx *sql.Context, r sql.Row) error {
return fmt.Errorf("the dolt_stashes table is read-only; use the dolt_stash stored procedure to edit remotes")
}
// Update the given row. Provides both the old and new rows.
func (bWr stashWriter) Update(ctx *sql.Context, old sql.Row, new sql.Row) error {
return fmt.Errorf("the dolt_stash table is read-only; use the dolt_stash stored procedure to edit remotes")
}
// Delete deletes the given row. Returns ErrDeleteRowNotFound if the row was not found. Delete will be called once for
// each row to process for the delete operation, which may involve many rows. After all rows have been processed,
// Close is called.
func (bWr stashWriter) Delete(ctx *sql.Context, r sql.Row) error {
return fmt.Errorf("the dolt_stash table is read-only; use the dolt_stash stored procedure to edit remotes")
}
// StatementBegin implements the interface sql.TableEditor. Currently a no-op.
func (bWr stashWriter) StatementBegin(ctx *sql.Context) {}
// DiscardChanges implements the interface sql.TableEditor. Currently a no-op.
func (bWr stashWriter) DiscardChanges(ctx *sql.Context, errorEncountered error) error {
return nil
}
// StatementComplete implements the interface sql.TableEditor. Currently a no-op.
func (bWr stashWriter) StatementComplete(ctx *sql.Context) error {
return nil
}
// Close finalizes the delete operation, persisting the result.
func (bWr stashWriter) Close(*sql.Context) error {
return nil
}

View File

@@ -2119,3 +2119,9 @@ func TestDoltHelpSystemTable(t *testing.T) {
defer harness.Close()
RunDoltHelpSystemTableTests(t, harness)
}
func TestDoltStash(t *testing.T) {
harness := newDoltEnginetestHarness(t)
defer harness.Close()
RunDoltStashSystemTableTests(t, harness)
}

View File

@@ -1992,3 +1992,13 @@ func RunDoltHelpSystemTableTests(t *testing.T, harness DoltEnginetestHarness) {
})
}
}
func RunDoltStashSystemTableTests(t *testing.T, h DoltEnginetestHarness) {
for _, script := range DoltStashTests {
func() {
h := h.NewHarness(t)
defer h.Close()
enginetest.TestScript(t, h, script)
}()
}
}

View File

@@ -8142,3 +8142,203 @@ end;
},
},
}
var DoltStashTests = []queries.ScriptTest{
{
Name: "DOLT_STASH() subcommands error on empty space.",
Assertions: []queries.ScriptTestAssertion{
{
Query: "CALL DOLT_STASH('push', 'myStash');",
ExpectedErrStr: "no local changes to save",
},
{
Query: "CALL DOLT_STASH('pop', 'myStash');",
ExpectedErrStr: "No stash entries found.",
},
{
Query: "CALL DOLT_STASH('drop', 'myStash');",
ExpectedErrStr: "No stash entries found.",
},
{
Query: "CALL DOLT_STASH('clear','myStash');",
Expected: []sql.Row{},
},
},
},
{
Name: "Simple push and pop with DOLT_STASH()",
SetUpScript: []string{
"CREATE TABLE test(pk BIGINT PRIMARY KEY, v varchar(10))",
"CALL DOLT_ADD('.')",
"CALL DOLT_COMMIT('-m', 'Created table')",
},
Assertions: []queries.ScriptTestAssertion{
{
Query: "INSERT INTO test VALUES (1, 'a');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "SELECT * FROM test",
Expected: []sql.Row{{1, 'a'}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "SELECT * FROM DOLT_STASHES;",
Expected: []sql.Row{{"stashes/myStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"}},
},
{
Query: "CALL DOLT_STASH('pop', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "SELECT * FROM DOLT_STASHES",
Expected: []sql.Row{},
},
},
},
{
Name: "Clearing stash removes all entries in stash list",
SetUpScript: []string{
"CREATE TABLE test(pk BIGINT PRIMARY KEY, v varchar(10))",
"CALL DOLT_ADD('.')",
"CALL DOLT_COMMIT('-m', 'Created table')",
},
Assertions: []queries.ScriptTestAssertion{
{
Query: "INSERT INTO test VALUES (1, 'a')",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "INSERT INTO test VALUES (2, 'b')",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'anotherStash');",
SkipResultsCheck: true,
},
{
Query: "INSERT INTO test VALUES (3, 'c')",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'anotherStash');",
SkipResultsCheck: true,
},
{
Query: "SELECT * FROM dolt_stashes;",
Expected: []sql.Row{
{"stashes/myStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"},
{"stashes/anotherStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"},
{"stashes/anotherStash", "stash@{1}", "refs/heads/main", doltCommit, "Created table"},
},
},
{
Query: "CALL DOLT_STASH('clear', 'anotherStash');",
SkipResultsCheck: true,
},
{
Query: "SELECT * FROM dolt_stashes;",
Expected: []sql.Row{
{"stashes/myStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"},
},
},
},
},
{
Name: "Clearing and stashing again",
SetUpScript: []string{
"CREATE TABLE test(pk BIGINT PRIMARY KEY, v varchar(10))",
"CALL DOLT_ADD('.')",
"CALL DOLT_COMMIT('-m', 'Created table')",
},
Assertions: []queries.ScriptTestAssertion{
{
Query: "INSERT INTO test VALUES (1, 'a');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "SELECT * FROM DOLT_STASHES;",
Expected: []sql.Row{
{"stashes/myStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"},
},
},
{
Query: "CALL DOLT_STASH('clear', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "INSERT INTO test VALUES (2, 'b');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "SELECT * FROM DOLT_STASHES;",
Expected: []sql.Row{
{"stashes/myStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"},
},
},
},
},
/*{
Name: "Popping specific stashes",
SetUpScript: []string{
"CREATE TABLE test(pk BIGINT PRIMARY KEY, v varchar(10))",
"CALL DOLT_ADD('.')",
"CALL DOLT_COMMIT('-m', 'Created table')",
},
Assertions: []queries.ScriptTestAssertion{
{
Query: "INSERT INTO test VALUES (1, 'a');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "INSERT INTO test VALUES (2, 'b');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "INSERT INTO test VALUES (3, 'c');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},{
Query: "INSERT INTO test VALUES (4, 'd');",
Expected: []sql.Row{{types.NewOkResult(1)}},
},
{
Query: "CALL DOLT_STASH('push', 'myStash');",
SkipResultsCheck: true,
},
{
Query: "CALL DOLT_STASH('pop', 'myStash', stash@{3}",
SkipResultsCheck: true
},
{
Query: "SELECT * FROM DOLT_STASHES",
Expected: []sql.Row{
{"stashes/myStash", "stash@{0}", "refs/heads/main", doltCommit, "Created table"},
},
},
},
},*/
}