mirror of
https://github.com/dolthub/dolt.git
synced 2026-01-28 18:59:00 -06:00
Make dolt_docs system table function like dolt_ignore
This commit is contained in:
@@ -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)
|
||||
|
||||
|
||||
@@ -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"
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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"
|
||||
|
||||
Reference in New Issue
Block a user