Make dolt_docs system table function like dolt_ignore

This commit is contained in:
Taylor Bantle
2023-10-16 16:02:13 -07:00
parent e3fd68c650
commit dbb0bf8e2b
7 changed files with 264 additions and 40 deletions

View File

@@ -151,10 +151,6 @@ func writeDocToTable(ctx context.Context, eng *engine.SqlEngine, dbName, docName
sctx.Session.SetClient(sql.Client{User: "root", Address: "%", Capabilities: 0})
if err = execQuery(sctx, eng, doltdb.DocsMaybeCreateTableStmt); err != nil {
return err
}
content = strings.ReplaceAll(content, `"`, `\"`)
update := fmt.Sprintf(writeDocTemplate, docName, content)

View File

@@ -202,13 +202,6 @@ const (
ReadmeDoc = "README.md"
)
var DocsMaybeCreateTableStmt = `
CREATE TABLE IF NOT EXISTS dolt_docs (
doc_name varchar(16383) NOT NULL,
doc_text longtext,
PRIMARY KEY (doc_name)
);`
const (
// DocTableName is the name of the dolt table containing documents such as the license and readme
DocTableName = "dolt_docs"

View File

@@ -453,6 +453,12 @@ func (db Database) getTableInsensitive(ctx *sql.Context, head *doltdb.Commit, ds
versionableTable := backingTable.(dtables.VersionableTable)
dt, found = dtables.NewIgnoreTable(ctx, versionableTable), true
}
case doltdb.DocTableName:
backingTable, _, err := db.getTable(ctx, root, doltdb.DocTableName)
if err != nil {
return nil, false, err
}
dt, found = dtables.NewDocsTable(ctx, backingTable), true
}
if found {
@@ -856,12 +862,7 @@ func (db Database) CreateTable(ctx *sql.Context, tableName string, sch sql.Prima
if err := dsess.CheckAccessForDb(ctx, db, branch_control.Permissions_Write); err != nil {
return err
}
if strings.ToLower(tableName) == doltdb.DocTableName {
// validate correct schema
if !dtables.DoltDocsSqlSchema.Equals(sch.Schema) && !dtables.OldDoltDocsSqlSchema.Equals(sch.Schema) {
return fmt.Errorf("incorrect schema for dolt_docs table")
}
} else if doltdb.HasDoltPrefix(tableName) && !doltdb.IsFullTextTable(tableName) {
if doltdb.HasDoltPrefix(tableName) && !doltdb.IsFullTextTable(tableName) {
return ErrReservedTableName.New(tableName)
}
@@ -877,12 +878,7 @@ func (db Database) CreateIndexedTable(ctx *sql.Context, tableName string, sch sq
if err := dsess.CheckAccessForDb(ctx, db, branch_control.Permissions_Write); err != nil {
return err
}
if strings.ToLower(tableName) == doltdb.DocTableName {
// validate correct schema
if !dtables.DoltDocsSqlSchema.Equals(sch.Schema) && !dtables.OldDoltDocsSqlSchema.Equals(sch.Schema) {
return fmt.Errorf("incorrect schema for dolt_docs table")
}
} else if doltdb.HasDoltPrefix(tableName) {
if doltdb.HasDoltPrefix(tableName) {
return ErrReservedTableName.New(tableName)
}

View File

@@ -15,10 +15,18 @@
package dtables
import (
"fmt"
"github.com/dolthub/go-mysql-server/sql"
sqlTypes "github.com/dolthub/go-mysql-server/sql/types"
"github.com/dolthub/vitess/go/sqltypes"
"github.com/dolthub/dolt/go/libraries/doltcore/doltdb"
"github.com/dolthub/dolt/go/libraries/doltcore/sqle/dsess"
"github.com/dolthub/dolt/go/libraries/doltcore/sqle/index"
"github.com/dolthub/dolt/go/libraries/doltcore/sqle/sqlutil"
"github.com/dolthub/dolt/go/libraries/doltcore/sqle/writer"
"github.com/dolthub/dolt/go/store/hash"
)
var DoltDocsSqlSchema sql.PrimaryKeySchema
@@ -28,3 +36,226 @@ func init() {
DoltDocsSqlSchema, _ = sqlutil.FromDoltSchema(doltdb.DocTableName, doltdb.DocsSchema)
OldDoltDocsSqlSchema, _ = sqlutil.FromDoltSchema(doltdb.DocTableName, doltdb.OldDocsSchema)
}
var _ sql.Table = (*DocsTable)(nil)
var _ sql.UpdatableTable = (*DocsTable)(nil)
var _ sql.DeletableTable = (*DocsTable)(nil)
var _ sql.InsertableTable = (*DocsTable)(nil)
var _ sql.ReplaceableTable = (*DocsTable)(nil)
// DocsTable is the system table that stores patterns for table names that should not be committed.
type DocsTable struct {
ddb *doltdb.DoltDB
backingTable sql.Table
}
func (i *DocsTable) Name() string {
return doltdb.DocTableName
}
func (i *DocsTable) String() string {
return doltdb.DocTableName
}
// Schema is a sql.Table interface function that gets the sql.Schema of the dolt_docs system table.
func (i *DocsTable) Schema() sql.Schema {
return []*sql.Column{
{Name: doltdb.DocPkColumnName, Type: sqlTypes.MustCreateString(sqltypes.VarChar, 16383, sql.Collation_Default), Source: doltdb.DocTableName, PrimaryKey: true, Nullable: false},
{Name: doltdb.DocTextColumnName, Type: sqlTypes.LongText, Source: doltdb.DocTableName, PrimaryKey: false},
}
}
func (i *DocsTable) Collation() sql.CollationID {
return sql.Collation_Default
}
// Partitions is a sql.Table interface function that returns a partition of the data.
func (i *DocsTable) Partitions(context *sql.Context) (sql.PartitionIter, error) {
if i.backingTable == nil {
// no backing table; return an empty iter.
return index.SinglePartitionIterFromNomsMap(nil), nil
}
return i.backingTable.Partitions(context)
}
func (i *DocsTable) PartitionRows(context *sql.Context, partition sql.Partition) (sql.RowIter, error) {
if i.backingTable == nil {
// no backing table; return an empty iter.
return sql.RowsToRowIter(), nil
}
return i.backingTable.PartitionRows(context, partition)
}
// NewDocsTable creates an DocsTable
func NewDocsTable(_ *sql.Context, ddb *doltdb.DoltDB, backingTable sql.Table) sql.Table {
return &DocsTable{ddb: ddb, backingTable: backingTable}
}
// 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 (it *DocsTable) Replacer(ctx *sql.Context) sql.RowReplacer {
return newDocsWriter(it)
}
// Updater returns a RowUpdater for this table. The RowUpdater will have Update called once for each row to be
// updated, followed by a call to Close() when all rows have been processed.
func (it *DocsTable) Updater(ctx *sql.Context) sql.RowUpdater {
return newDocsWriter(it)
}
// Inserter returns an Inserter for this table. The Inserter will get one call to Insert() for each row to be
// inserted, and will end with a call to Close() to finalize the insert operation.
func (it *DocsTable) Inserter(*sql.Context) sql.RowInserter {
return newDocsWriter(it)
}
// 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 (it *DocsTable) Deleter(*sql.Context) sql.RowDeleter {
return newDocsWriter(it)
}
var _ sql.RowReplacer = (*docsWriter)(nil)
var _ sql.RowUpdater = (*docsWriter)(nil)
var _ sql.RowInserter = (*docsWriter)(nil)
var _ sql.RowDeleter = (*docsWriter)(nil)
type docsWriter struct {
it *DocsTable
errDuringStatementBegin error
prevHash *hash.Hash
tableWriter writer.TableWriter
}
func newDocsWriter(it *DocsTable) *docsWriter {
return &docsWriter{it, nil, nil, nil}
}
// 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 (iw *docsWriter) Insert(ctx *sql.Context, r sql.Row) error {
if err := iw.errDuringStatementBegin; err != nil {
return err
}
return iw.tableWriter.Insert(ctx, r)
}
// Update the given row. Provides both the old and new rows.
func (iw *docsWriter) Update(ctx *sql.Context, old sql.Row, new sql.Row) error {
if err := iw.errDuringStatementBegin; err != nil {
return err
}
return iw.tableWriter.Update(ctx, old, new)
}
// 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 (iw *docsWriter) Delete(ctx *sql.Context, r sql.Row) error {
if err := iw.errDuringStatementBegin; err != nil {
return err
}
return iw.tableWriter.Delete(ctx, r)
}
// StatementBegin is called before the first operation of a statement. Integrators should mark the state of the data
// in some way that it may be returned to in the case of an error.
func (iw *docsWriter) StatementBegin(ctx *sql.Context) {
dbName := ctx.GetCurrentDatabase()
dSess := dsess.DSessFromSess(ctx.Session)
// TODO: this needs to use a revision qualified name
roots, _ := dSess.GetRoots(ctx, dbName)
dbState, ok, err := dSess.LookupDbState(ctx, dbName)
if err != nil {
iw.errDuringStatementBegin = err
return
}
if !ok {
iw.errDuringStatementBegin = fmt.Errorf("no root value found in session")
return
}
prevHash, err := roots.Working.HashOf()
if err != nil {
iw.errDuringStatementBegin = err
return
}
iw.prevHash = &prevHash
found, err := roots.Working.HasTable(ctx, doltdb.DocTableName)
if err != nil {
iw.errDuringStatementBegin = err
return
}
if !found {
// TODO: This is effectively a duplicate of the schema declaration above in a different format.
// We should find a way to not repeat ourselves.
newSchema := doltdb.DocsSchema
// underlying table doesn't exist. Record this, then create the table.
newRootValue, err := roots.Working.CreateEmptyTable(ctx, doltdb.DocTableName, newSchema)
if err != nil {
iw.errDuringStatementBegin = err
return
}
if dbState.WorkingSet() == nil {
iw.errDuringStatementBegin = doltdb.ErrOperationNotSupportedInDetachedHead
return
}
// We use WriteSession.SetWorkingSet instead of DoltSession.SetRoot because we want to avoid modifying the root
// until the end of the transaction, but we still want the WriteSession to be able to find the newly
// created table.
err = dbState.WriteSession().SetWorkingSet(ctx, dbState.WorkingSet().WithWorkingRoot(newRootValue))
if err != nil {
iw.errDuringStatementBegin = err
return
}
err = dSess.SetRoot(ctx, dbName, newRootValue)
if err != nil {
iw.errDuringStatementBegin = err
return
}
}
tableWriter, err := dbState.WriteSession().GetTableWriter(ctx, doltdb.DocTableName, dbName, dSess.SetRoot)
if err != nil {
iw.errDuringStatementBegin = err
return
}
iw.tableWriter = tableWriter
tableWriter.StatementBegin(ctx)
}
// DiscardChanges is called if a statement encounters an error, and all current changes since the statement beginning
// should be discarded.
func (iw *docsWriter) DiscardChanges(ctx *sql.Context, errorEncountered error) error {
if iw.tableWriter != nil {
return iw.tableWriter.DiscardChanges(ctx, errorEncountered)
}
return nil
}
// StatementComplete is called after the last operation of the statement, indicating that it has successfully completed.
// The mark set in StatementBegin may be removed, and a new one should be created on the next StatementBegin.
func (iw *docsWriter) StatementComplete(ctx *sql.Context) error {
return iw.tableWriter.StatementComplete(ctx)
}
// Close finalizes the delete operation, persisting the result.
func (iw docsWriter) Close(ctx *sql.Context) error {
if iw.tableWriter != nil {
return iw.tableWriter.Close(ctx)
}
return nil
}

View File

@@ -805,8 +805,8 @@ func TestRenameTableStatements(t *testing.T) {
}
func TestAlterSystemTables(t *testing.T) {
systemTableNames := []string{"dolt_log", "dolt_history_people", "dolt_diff_people", "dolt_commit_diff_people", "dolt_schemas"} // "dolt_docs",
reservedTableNames := []string{"dolt_query_catalog"}
systemTableNames := []string{"dolt_log", "dolt_history_people", "dolt_diff_people", "dolt_commit_diff_people", "dolt_schemas"}
reservedTableNames := []string{"dolt_query_catalog", "dolt_docs", "dolt_procedures", "dolt_ignore"}
var dEnv *env.DoltEnv
var err error
@@ -814,18 +814,15 @@ func TestAlterSystemTables(t *testing.T) {
dEnv, err = CreateTestDatabase()
require.NoError(t, err)
err := CreateEmptyTestTable(dEnv, "dolt_docs", doltdb.DocsSchema)
require.NoError(t, err)
err = CreateEmptyTestTable(dEnv, doltdb.SchemasTableName, schemaTableSchema)
require.NoError(t, err)
CreateTestTable(t, dEnv, "dolt_docs", doltdb.DocsSchema,
"INSERT INTO dolt_docs VALUES ('LICENSE.md','A license')")
CreateTestTable(t, dEnv, doltdb.DoltQueryCatalogTableName, dtables.DoltQueryCatalogSchema,
"INSERT INTO dolt_query_catalog VALUES ('abc123', 1, 'example', 'select 2+2 from dual', 'description')")
CreateTestTable(t, dEnv, doltdb.SchemasTableName, schemaTableSchema,
"INSERT INTO dolt_schemas (type, name, fragment) VALUES ('view', 'name', 'create view name as select 2+2 from dual')")
ExecuteSetupSQL(context.Background(), `
CREATE PROCEDURE simple_proc2() SELECT 1+1;
INSERT INTO dolt_ignore VALUES ('test', 1);`)(t, dEnv)
}
t.Run("Create", func(t *testing.T) {

View File

@@ -983,6 +983,14 @@ var AsOfTests = []SelectTest{
Query: "select * from test_table as of CONVERT('1970-01-01 02:00:00', DATETIME)",
ExpectedErr: "not found",
},
{
Name: "select from dolt_docs as of main",
AdditionalSetup: CreateTableFn("dolt_docs", doltdb.DocsSchema,
"INSERT INTO dolt_docs VALUES ('LICENSE.md','A license')"),
Query: "select * from dolt_docs as of 'main'",
ExpectedRows: []sql.Row{{"LICENSE.md", "A license"}},
ExpectedSchema: CompressSchema(doltdb.DocsSchema),
},
}
// Tests of join functionality, basically any query involving more than one table should go here for now.

View File

@@ -94,6 +94,9 @@ teardown() {
}
@test "docs: docs are available from SQL" {
run dolt sql -q "SELECT * FROM dolt_docs"
[ "$status" -eq 0 ]
dolt docs upload LICENSE.md LICENSE.md
dolt sql -q "SELECT doc_name FROM dolt_docs" -r csv
run dolt sql -q "SELECT doc_name FROM dolt_docs" -r csv
@@ -103,18 +106,18 @@ teardown() {
}
@test "docs: docs can be created from SQL" {
# must use correct schema
run dolt sql -q "CREATE TABLE dolt_docs (x int);"
[ "$status" -ne 0 ]
[[ "$output" =~ "incorrect schema for dolt_docs table" ]] || false
[[ "$output" =~ "Invalid table name dolt_docs" ]] || false
run dolt sql -q "CREATE TABLE dolt_docs (
doc_name varchar(16383) NOT NULL,
doc_text longtext,
PRIMARY KEY (doc_name)
);"
[ "$status" -ne 0 ]
[[ "$output" =~ "Invalid table name dolt_docs" ]] || false
dolt sql <<SQL
CREATE TABLE dolt_docs (
doc_name varchar(16383) NOT NULL,
doc_text longtext,
PRIMARY KEY (doc_name)
);
SQL
dolt sql -q "INSERT INTO dolt_docs VALUES ('README.md', 'this is a README')"
run dolt sql -q "SELECT * FROM dolt_docs"