mirror of https://github.com/citusdata/citus.git
Propagate REINDEX TABLE & REINDEX INDEX
parent
66b9f2e887
commit
363409a0c2
|
@ -19,6 +19,7 @@
|
||||||
#include "catalog/index.h"
|
#include "catalog/index.h"
|
||||||
#include "catalog/namespace.h"
|
#include "catalog/namespace.h"
|
||||||
#include "catalog/pg_class.h"
|
#include "catalog/pg_class.h"
|
||||||
|
#include "commands/tablecmds.h"
|
||||||
#include "distributed/citus_ruleutils.h"
|
#include "distributed/citus_ruleutils.h"
|
||||||
#include "distributed/commands.h"
|
#include "distributed/commands.h"
|
||||||
#include "distributed/commands/utility_hook.h"
|
#include "distributed/commands/utility_hook.h"
|
||||||
|
@ -39,8 +40,12 @@
|
||||||
|
|
||||||
/* Local functions forward declarations for helper functions */
|
/* Local functions forward declarations for helper functions */
|
||||||
static List * CreateIndexTaskList(Oid relationId, IndexStmt *indexStmt);
|
static List * CreateIndexTaskList(Oid relationId, IndexStmt *indexStmt);
|
||||||
|
static List * CreateReindexTaskList(Oid relationId, ReindexStmt *reindexStmt);
|
||||||
static void RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid,
|
static void RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid,
|
||||||
void *arg);
|
void *arg);
|
||||||
|
static void RangeVarCallbackForReindexIndex(const RangeVar *rel, Oid relOid, Oid
|
||||||
|
oldRelOid,
|
||||||
|
void *arg);
|
||||||
static void ErrorIfUnsupportedIndexStmt(IndexStmt *createIndexStatement);
|
static void ErrorIfUnsupportedIndexStmt(IndexStmt *createIndexStatement);
|
||||||
static void ErrorIfUnsupportedDropIndexStmt(DropStmt *dropIndexStatement);
|
static void ErrorIfUnsupportedDropIndexStmt(DropStmt *dropIndexStatement);
|
||||||
static List * DropIndexTaskList(Oid relationId, Oid indexId, DropStmt *dropStmt);
|
static List * DropIndexTaskList(Oid relationId, Oid indexId, DropStmt *dropStmt);
|
||||||
|
@ -57,6 +62,18 @@ struct DropRelationCallbackState
|
||||||
bool concurrent;
|
bool concurrent;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This struct defines the state for the callback for reindex statements.
|
||||||
|
* It is copied as it is from commands/indexcmds.c in Postgres source.
|
||||||
|
*/
|
||||||
|
struct ReindexIndexCallbackState
|
||||||
|
{
|
||||||
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
bool concurrent;
|
||||||
|
#endif
|
||||||
|
Oid locked_table_oid;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* IsIndexRenameStmt returns whether the passed-in RenameStmt is the following
|
* IsIndexRenameStmt returns whether the passed-in RenameStmt is the following
|
||||||
|
@ -101,7 +118,6 @@ PlanIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand)
|
||||||
Relation relation = NULL;
|
Relation relation = NULL;
|
||||||
Oid relationId = InvalidOid;
|
Oid relationId = InvalidOid;
|
||||||
bool isDistributedRelation = false;
|
bool isDistributedRelation = false;
|
||||||
char *namespaceName = NULL;
|
|
||||||
LOCKMODE lockmode = ShareLock;
|
LOCKMODE lockmode = ShareLock;
|
||||||
MemoryContext relationContext = NULL;
|
MemoryContext relationContext = NULL;
|
||||||
|
|
||||||
|
@ -126,6 +142,8 @@ PlanIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand)
|
||||||
|
|
||||||
isDistributedRelation = IsDistributedTable(relationId);
|
isDistributedRelation = IsDistributedTable(relationId);
|
||||||
|
|
||||||
|
if (createIndexStatement->relation->schemaname == NULL)
|
||||||
|
{
|
||||||
/*
|
/*
|
||||||
* Before we do any further processing, fix the schema name to make sure
|
* Before we do any further processing, fix the schema name to make sure
|
||||||
* that a (distributed) table with the same name does not appear on the
|
* that a (distributed) table with the same name does not appear on the
|
||||||
|
@ -133,12 +151,13 @@ PlanIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand)
|
||||||
* table is not distributed, since a distributed table may appear on the
|
* table is not distributed, since a distributed table may appear on the
|
||||||
* search path by the time postgres starts processing this statement.
|
* search path by the time postgres starts processing this statement.
|
||||||
*/
|
*/
|
||||||
namespaceName = get_namespace_name(RelationGetNamespace(relation));
|
char *namespaceName = get_namespace_name(RelationGetNamespace(relation));
|
||||||
|
|
||||||
/* ensure we copy string into proper context */
|
/* ensure we copy string into proper context */
|
||||||
relationContext = GetMemoryChunkContext(createIndexStatement->relation);
|
relationContext = GetMemoryChunkContext(createIndexStatement->relation);
|
||||||
namespaceName = MemoryContextStrdup(relationContext, namespaceName);
|
createIndexStatement->relation->schemaname = MemoryContextStrdup(
|
||||||
createIndexStatement->relation->schemaname = namespaceName;
|
relationContext, namespaceName);
|
||||||
|
}
|
||||||
|
|
||||||
heap_close(relation, NoLock);
|
heap_close(relation, NoLock);
|
||||||
|
|
||||||
|
@ -147,6 +166,7 @@ PlanIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand)
|
||||||
Oid namespaceId = InvalidOid;
|
Oid namespaceId = InvalidOid;
|
||||||
Oid indexRelationId = InvalidOid;
|
Oid indexRelationId = InvalidOid;
|
||||||
char *indexName = createIndexStatement->idxname;
|
char *indexName = createIndexStatement->idxname;
|
||||||
|
char *namespaceName = createIndexStatement->relation->schemaname;
|
||||||
|
|
||||||
ErrorIfUnsupportedIndexStmt(createIndexStatement);
|
ErrorIfUnsupportedIndexStmt(createIndexStatement);
|
||||||
|
|
||||||
|
@ -172,51 +192,84 @@ PlanIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand)
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ErrorIfReindexOnDistributedTable determines whether a given REINDEX
|
* PlanReindexStmt determines whether a given REINDEX statement involves
|
||||||
* involves a distributed table, & raises an error if so.
|
* a distributed table. If so (and if the statement does not use unsupported
|
||||||
|
* options), it modifies the input statement to ensure proper execution against
|
||||||
|
* the master node table and creates a DDLJob to encapsulate information needed
|
||||||
|
* during the worker node portion of DDL execution before returning that DDLJob
|
||||||
|
* in a List. If no distributed table is involved, this function returns NIL.
|
||||||
*/
|
*/
|
||||||
void
|
List *
|
||||||
ErrorIfReindexOnDistributedTable(ReindexStmt *ReindexStatement)
|
PlanReindexStmt(ReindexStmt *reindexStatement, const char *reindexCommand)
|
||||||
|
{
|
||||||
|
List *ddlJobs = NIL;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We first check whether a distributed relation is affected. For that, we need to
|
||||||
|
* open the relation. To prevent race conditions with later lookups, lock the table,
|
||||||
|
* and modify the rangevar to include the schema.
|
||||||
|
*/
|
||||||
|
if (reindexStatement->relation != NULL)
|
||||||
{
|
{
|
||||||
Relation relation = NULL;
|
Relation relation = NULL;
|
||||||
Oid relationId = InvalidOid;
|
Oid relationId = InvalidOid;
|
||||||
bool isDistributedRelation = false;
|
bool isDistributedRelation = false;
|
||||||
LOCKMODE lockmode = AccessShareLock;
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
LOCKMODE lockmode = reindexStatement->concurrent ? ShareUpdateExclusiveLock :
|
||||||
|
AccessExclusiveLock;
|
||||||
|
#else
|
||||||
|
LOCKMODE lockmode = AccessExclusiveLock;
|
||||||
|
#endif
|
||||||
|
MemoryContext relationContext = NULL;
|
||||||
|
|
||||||
/*
|
Assert(reindexStatement->kind == REINDEX_OBJECT_INDEX ||
|
||||||
* We first check whether a distributed relation is affected. For that, we need to
|
reindexStatement->kind == REINDEX_OBJECT_TABLE);
|
||||||
* open the relation.
|
|
||||||
*/
|
if (reindexStatement->kind == REINDEX_OBJECT_INDEX)
|
||||||
if (ReindexStatement->relation == NULL)
|
|
||||||
{
|
{
|
||||||
/* ignore REINDEX SCHEMA, REINDEX SYSTEM, and REINDEX DATABASE */
|
Oid indOid;
|
||||||
return;
|
struct ReindexIndexCallbackState state;
|
||||||
}
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
state.concurrent = reindexStatement->concurrent;
|
||||||
|
#endif
|
||||||
|
state.locked_table_oid = InvalidOid;
|
||||||
|
|
||||||
Assert(ReindexStatement->kind == REINDEX_OBJECT_INDEX ||
|
indOid = RangeVarGetRelidInternal(reindexStatement->relation,
|
||||||
ReindexStatement->kind == REINDEX_OBJECT_TABLE);
|
lockmode, 0,
|
||||||
|
RangeVarCallbackForReindexIndex,
|
||||||
/*
|
&state);
|
||||||
* XXX: Consider using RangeVarGetRelidExtended with a permission
|
relation = index_open(indOid, NoLock);
|
||||||
* checking callback. Right now we'll acquire the lock before having
|
|
||||||
* checked permissions.
|
|
||||||
*/
|
|
||||||
if (ReindexStatement->kind == REINDEX_OBJECT_INDEX)
|
|
||||||
{
|
|
||||||
Oid indOid = RangeVarGetRelid(ReindexStatement->relation,
|
|
||||||
NoLock, false);
|
|
||||||
relation = index_open(indOid, lockmode);
|
|
||||||
relationId = IndexGetRelation(indOid, false);
|
relationId = IndexGetRelation(indOid, false);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
relation = heap_openrv(ReindexStatement->relation, lockmode);
|
RangeVarGetRelidInternal(reindexStatement->relation, lockmode, 0,
|
||||||
|
RangeVarCallbackOwnsTable, NULL);
|
||||||
|
|
||||||
|
relation = heap_openrv(reindexStatement->relation, NoLock);
|
||||||
relationId = RelationGetRelid(relation);
|
relationId = RelationGetRelid(relation);
|
||||||
}
|
}
|
||||||
|
|
||||||
isDistributedRelation = IsDistributedTable(relationId);
|
isDistributedRelation = IsDistributedTable(relationId);
|
||||||
|
|
||||||
if (ReindexStatement->kind == REINDEX_OBJECT_INDEX)
|
if (reindexStatement->relation->schemaname == NULL)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Before we do any further processing, fix the schema name to make sure
|
||||||
|
* that a (distributed) table with the same name does not appear on the
|
||||||
|
* search path in front of the current schema. We do this even if the
|
||||||
|
* table is not distributed, since a distributed table may appear on the
|
||||||
|
* search path by the time postgres starts processing this statement.
|
||||||
|
*/
|
||||||
|
char *namespaceName = get_namespace_name(RelationGetNamespace(relation));
|
||||||
|
|
||||||
|
/* ensure we copy string into proper context */
|
||||||
|
relationContext = GetMemoryChunkContext(reindexStatement->relation);
|
||||||
|
reindexStatement->relation->schemaname = MemoryContextStrdup(relationContext,
|
||||||
|
namespaceName);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (reindexStatement->kind == REINDEX_OBJECT_INDEX)
|
||||||
{
|
{
|
||||||
index_close(relation, NoLock);
|
index_close(relation, NoLock);
|
||||||
}
|
}
|
||||||
|
@ -227,12 +280,23 @@ ErrorIfReindexOnDistributedTable(ReindexStmt *ReindexStatement)
|
||||||
|
|
||||||
if (isDistributedRelation)
|
if (isDistributedRelation)
|
||||||
{
|
{
|
||||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
DDLJob *ddlJob = palloc0(sizeof(DDLJob));
|
||||||
errmsg(
|
ddlJob->targetRelationId = relationId;
|
||||||
"REINDEX is not implemented for distributed relations")));
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
ddlJob->concurrentIndexCmd = reindexStatement->concurrent;
|
||||||
|
#else
|
||||||
|
ddlJob->concurrentIndexCmd = false;
|
||||||
|
#endif
|
||||||
|
ddlJob->commandString = reindexCommand;
|
||||||
|
ddlJob->taskList = CreateReindexTaskList(relationId, reindexStatement);
|
||||||
|
|
||||||
|
ddlJobs = list_make1(ddlJob);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return ddlJobs;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* PlanDropIndexStmt determines whether a given DROP INDEX statement involves
|
* PlanDropIndexStmt determines whether a given DROP INDEX statement involves
|
||||||
|
@ -487,6 +551,52 @@ CreateIndexTaskList(Oid relationId, IndexStmt *indexStmt)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CreateReindexTaskList builds a list of tasks to execute a REINDEX command
|
||||||
|
* against a specified distributed table.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
CreateReindexTaskList(Oid relationId, ReindexStmt *reindexStmt)
|
||||||
|
{
|
||||||
|
List *taskList = NIL;
|
||||||
|
List *shardIntervalList = LoadShardIntervalList(relationId);
|
||||||
|
ListCell *shardIntervalCell = NULL;
|
||||||
|
StringInfoData ddlString;
|
||||||
|
uint64 jobId = INVALID_JOB_ID;
|
||||||
|
int taskId = 1;
|
||||||
|
|
||||||
|
initStringInfo(&ddlString);
|
||||||
|
|
||||||
|
/* lock metadata before getting placement lists */
|
||||||
|
LockShardListMetadata(shardIntervalList, ShareLock);
|
||||||
|
|
||||||
|
foreach(shardIntervalCell, shardIntervalList)
|
||||||
|
{
|
||||||
|
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
|
||||||
|
uint64 shardId = shardInterval->shardId;
|
||||||
|
Task *task = NULL;
|
||||||
|
|
||||||
|
deparse_shard_reindex_statement(reindexStmt, relationId, shardId, &ddlString);
|
||||||
|
|
||||||
|
task = CitusMakeNode(Task);
|
||||||
|
task->jobId = jobId;
|
||||||
|
task->taskId = taskId++;
|
||||||
|
task->taskType = DDL_TASK;
|
||||||
|
task->queryString = pstrdup(ddlString.data);
|
||||||
|
task->replicationModel = REPLICATION_MODEL_INVALID;
|
||||||
|
task->dependedTaskList = NULL;
|
||||||
|
task->anchorShardId = shardId;
|
||||||
|
task->taskPlacementList = FinalizedShardPlacementList(shardId);
|
||||||
|
|
||||||
|
taskList = lappend(taskList, task);
|
||||||
|
|
||||||
|
resetStringInfo(&ddlString);
|
||||||
|
}
|
||||||
|
|
||||||
|
return taskList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Before acquiring a table lock, check whether we have sufficient rights.
|
* Before acquiring a table lock, check whether we have sufficient rights.
|
||||||
* In the case of DROP INDEX, also try to lock the table before the index.
|
* In the case of DROP INDEX, also try to lock the table before the index.
|
||||||
|
@ -585,6 +695,90 @@ RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid, voi
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Check permissions on table before acquiring relation lock; also lock
|
||||||
|
* the heap before the RangeVarGetRelidExtended takes the index lock, to avoid
|
||||||
|
* deadlocks.
|
||||||
|
*
|
||||||
|
* This code is borrowed from RangeVarCallbackForReindexIndex() in
|
||||||
|
* commands/indexcmds.c in Postgres source. We need this to ensure the right
|
||||||
|
* order of locking while dealing with REINDEX statements.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
RangeVarCallbackForReindexIndex(const RangeVar *relation, Oid relId, Oid oldRelId,
|
||||||
|
void *arg)
|
||||||
|
{
|
||||||
|
/* *INDENT-OFF* */
|
||||||
|
char relkind;
|
||||||
|
struct ReindexIndexCallbackState *state = arg;
|
||||||
|
LOCKMODE table_lockmode;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Lock level here should match table lock in reindex_index() for
|
||||||
|
* non-concurrent case and table locks used by index_concurrently_*() for
|
||||||
|
* concurrent case.
|
||||||
|
*/
|
||||||
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
table_lockmode = state->concurrent ? ShareUpdateExclusiveLock : ShareLock;
|
||||||
|
#else
|
||||||
|
table_lockmode = ShareLock;
|
||||||
|
#endif
|
||||||
|
|
||||||
|
/*
|
||||||
|
* If we previously locked some other index's heap, and the name we're
|
||||||
|
* looking up no longer refers to that relation, release the now-useless
|
||||||
|
* lock.
|
||||||
|
*/
|
||||||
|
if (relId != oldRelId && OidIsValid(oldRelId))
|
||||||
|
{
|
||||||
|
UnlockRelationOid(state->locked_table_oid, table_lockmode);
|
||||||
|
state->locked_table_oid = InvalidOid;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* If the relation does not exist, there's nothing more to do. */
|
||||||
|
if (!OidIsValid(relId))
|
||||||
|
return;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* If the relation does exist, check whether it's an index. But note that
|
||||||
|
* the relation might have been dropped between the time we did the name
|
||||||
|
* lookup and now. In that case, there's nothing to do.
|
||||||
|
*/
|
||||||
|
relkind = get_rel_relkind(relId);
|
||||||
|
if (!relkind)
|
||||||
|
return;
|
||||||
|
if (relkind != RELKIND_INDEX
|
||||||
|
#if PG_VERSION_NUM >= 110000
|
||||||
|
&& relkind != RELKIND_PARTITIONED_INDEX
|
||||||
|
#endif
|
||||||
|
)
|
||||||
|
ereport(ERROR,
|
||||||
|
(errcode(ERRCODE_WRONG_OBJECT_TYPE),
|
||||||
|
errmsg("\"%s\" is not an index", relation->relname)));
|
||||||
|
|
||||||
|
/* Check permissions */
|
||||||
|
if (!pg_class_ownercheck(relId, GetUserId()))
|
||||||
|
aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_INDEX, relation->relname);
|
||||||
|
|
||||||
|
/* Lock heap before index to avoid deadlock. */
|
||||||
|
if (relId != oldRelId)
|
||||||
|
{
|
||||||
|
Oid table_oid = IndexGetRelation(relId, true);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* If the OID isn't valid, it means the index was concurrently
|
||||||
|
* dropped, which is not a problem for us; just return normally.
|
||||||
|
*/
|
||||||
|
if (OidIsValid(table_oid))
|
||||||
|
{
|
||||||
|
LockRelationOid(table_oid, table_lockmode);
|
||||||
|
state->locked_table_oid = table_oid;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
/* *INDENT-ON* */
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ErrorIfUnsupportedIndexStmt checks if the corresponding index statement is
|
* ErrorIfUnsupportedIndexStmt checks if the corresponding index statement is
|
||||||
* supported for distributed tables and errors out if it is not.
|
* supported for distributed tables and errors out if it is not.
|
||||||
|
|
|
@ -386,7 +386,7 @@ multi_ProcessUtility(PlannedStmt *pstmt,
|
||||||
|
|
||||||
if (IsA(parsetree, ReindexStmt))
|
if (IsA(parsetree, ReindexStmt))
|
||||||
{
|
{
|
||||||
ErrorIfReindexOnDistributedTable((ReindexStmt *) parsetree);
|
ddlJobs = PlanReindexStmt((ReindexStmt *) parsetree, queryString);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (IsA(parsetree, DropStmt))
|
if (IsA(parsetree, DropStmt))
|
||||||
|
|
|
@ -406,15 +406,6 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
|
||||||
|
|
||||||
AppendShardIdToName(objectName, shardId);
|
AppendShardIdToName(objectName, shardId);
|
||||||
}
|
}
|
||||||
else if (objectType == REINDEX_OBJECT_DATABASE)
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("cannot extend name for multi-relation reindex")));
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("invalid object type in reindex statement"),
|
|
||||||
errdetail("Object type: %u", (uint32) objectType)));
|
|
||||||
}
|
|
||||||
|
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,6 +38,7 @@
|
||||||
#include "distributed/multi_partitioning_utils.h"
|
#include "distributed/multi_partitioning_utils.h"
|
||||||
#include "distributed/relay_utility.h"
|
#include "distributed/relay_utility.h"
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/version_compat.h"
|
#include "distributed/version_compat.h"
|
||||||
#include "foreign/foreign.h"
|
#include "foreign/foreign.h"
|
||||||
#include "lib/stringinfo.h"
|
#include "lib/stringinfo.h"
|
||||||
|
@ -715,6 +716,82 @@ deparse_shard_index_statement(IndexStmt *origStmt, Oid distrelid, int64 shardid,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* deparse_shard_reindex_statement uses the provided REINDEX node, dist.
|
||||||
|
* relation, and shard identifier to populate a provided buffer with a string
|
||||||
|
* representation of a shard-extended version of that command.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
deparse_shard_reindex_statement(ReindexStmt *origStmt, Oid distrelid, int64 shardid,
|
||||||
|
StringInfo buffer)
|
||||||
|
{
|
||||||
|
ReindexStmt *reindexStmt = copyObject(origStmt); /* copy to avoid modifications */
|
||||||
|
char *relationName = NULL;
|
||||||
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
const char *concurrentlyString = reindexStmt->concurrent ? "CONCURRENTLY " : "";
|
||||||
|
#else
|
||||||
|
const char *concurrentlyString = "";
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
|
if (reindexStmt->kind == REINDEX_OBJECT_INDEX ||
|
||||||
|
reindexStmt->kind == REINDEX_OBJECT_TABLE)
|
||||||
|
{
|
||||||
|
relationName = reindexStmt->relation->relname;
|
||||||
|
|
||||||
|
/* extend relation and index name using shard identifier */
|
||||||
|
AppendShardIdToName(&relationName, shardid);
|
||||||
|
}
|
||||||
|
|
||||||
|
appendStringInfoString(buffer, "REINDEX ");
|
||||||
|
|
||||||
|
if (reindexStmt->options == REINDEXOPT_VERBOSE)
|
||||||
|
{
|
||||||
|
appendStringInfoString(buffer, "(VERBOSE) ");
|
||||||
|
}
|
||||||
|
|
||||||
|
switch (reindexStmt->kind)
|
||||||
|
{
|
||||||
|
case REINDEX_OBJECT_INDEX:
|
||||||
|
{
|
||||||
|
appendStringInfo(buffer, "INDEX %s%s", concurrentlyString,
|
||||||
|
quote_qualified_identifier(reindexStmt->relation->schemaname,
|
||||||
|
relationName));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case REINDEX_OBJECT_TABLE:
|
||||||
|
{
|
||||||
|
appendStringInfo(buffer, "TABLE %s%s", concurrentlyString,
|
||||||
|
quote_qualified_identifier(reindexStmt->relation->schemaname,
|
||||||
|
relationName));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case REINDEX_OBJECT_SCHEMA:
|
||||||
|
{
|
||||||
|
appendStringInfo(buffer, "SCHEMA %s%s", concurrentlyString,
|
||||||
|
quote_identifier(reindexStmt->name));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case REINDEX_OBJECT_SYSTEM:
|
||||||
|
{
|
||||||
|
appendStringInfo(buffer, "SYSTEM %s%s", concurrentlyString,
|
||||||
|
quote_identifier(reindexStmt->name));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
case REINDEX_OBJECT_DATABASE:
|
||||||
|
{
|
||||||
|
appendStringInfo(buffer, "DATABASE %s%s", concurrentlyString,
|
||||||
|
quote_identifier(reindexStmt->name));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/* deparse_index_columns appends index or include parameters to the provided buffer */
|
/* deparse_index_columns appends index or include parameters to the provided buffer */
|
||||||
static void
|
static void
|
||||||
deparse_index_columns(StringInfo buffer, List *indexParameterList, List *deparseContext)
|
deparse_index_columns(StringInfo buffer, List *indexParameterList, List *deparseContext)
|
||||||
|
|
|
@ -35,6 +35,8 @@ extern void EnsureRelationKindSupported(Oid relationId);
|
||||||
extern char * pg_get_tablecolumnoptionsdef_string(Oid tableRelationId);
|
extern char * pg_get_tablecolumnoptionsdef_string(Oid tableRelationId);
|
||||||
extern void deparse_shard_index_statement(IndexStmt *origStmt, Oid distrelid,
|
extern void deparse_shard_index_statement(IndexStmt *origStmt, Oid distrelid,
|
||||||
int64 shardid, StringInfo buffer);
|
int64 shardid, StringInfo buffer);
|
||||||
|
extern void deparse_shard_reindex_statement(ReindexStmt *origStmt, Oid distrelid,
|
||||||
|
int64 shardid, StringInfo buffer);
|
||||||
extern char * pg_get_indexclusterdef_string(Oid indexRelationId);
|
extern char * pg_get_indexclusterdef_string(Oid indexRelationId);
|
||||||
extern List * pg_get_table_grants(Oid relationId);
|
extern List * pg_get_table_grants(Oid relationId);
|
||||||
extern bool contain_nextval_expression_walker(Node *node, void *context);
|
extern bool contain_nextval_expression_walker(Node *node, void *context);
|
||||||
|
|
|
@ -58,7 +58,8 @@ extern List * PlanGrantStmt(GrantStmt *grantStmt);
|
||||||
extern bool IsIndexRenameStmt(RenameStmt *renameStmt);
|
extern bool IsIndexRenameStmt(RenameStmt *renameStmt);
|
||||||
extern List * PlanIndexStmt(IndexStmt *createIndexStatement,
|
extern List * PlanIndexStmt(IndexStmt *createIndexStatement,
|
||||||
const char *createIndexCommand);
|
const char *createIndexCommand);
|
||||||
extern void ErrorIfReindexOnDistributedTable(ReindexStmt *ReindexStatement);
|
extern List * PlanReindexStmt(ReindexStmt *ReindexStatement,
|
||||||
|
const char *ReindexCommand);
|
||||||
extern List * PlanDropIndexStmt(DropStmt *dropIndexStatement,
|
extern List * PlanDropIndexStmt(DropStmt *dropIndexStatement,
|
||||||
const char *dropIndexCommand);
|
const char *dropIndexCommand);
|
||||||
extern void PostProcessIndexStmt(IndexStmt *indexStmt);
|
extern void PostProcessIndexStmt(IndexStmt *indexStmt);
|
||||||
|
|
|
@ -207,17 +207,11 @@ SELECT * FROM pg_indexes WHERE tablename = 'lineitem' or tablename like 'index_t
|
||||||
--
|
--
|
||||||
-- REINDEX
|
-- REINDEX
|
||||||
--
|
--
|
||||||
SET citus.log_remote_commands to on;
|
|
||||||
SET client_min_messages = LOG;
|
|
||||||
REINDEX INDEX lineitem_orderkey_index;
|
REINDEX INDEX lineitem_orderkey_index;
|
||||||
ERROR: REINDEX is not implemented for distributed relations
|
|
||||||
REINDEX TABLE lineitem;
|
REINDEX TABLE lineitem;
|
||||||
ERROR: REINDEX is not implemented for distributed relations
|
|
||||||
REINDEX SCHEMA public;
|
REINDEX SCHEMA public;
|
||||||
REINDEX DATABASE regression;
|
REINDEX DATABASE regression;
|
||||||
REINDEX SYSTEM regression;
|
REINDEX SYSTEM regression;
|
||||||
SET citus.log_remote_commands to off;
|
|
||||||
RESET client_min_messages;
|
|
||||||
--
|
--
|
||||||
-- DROP INDEX
|
-- DROP INDEX
|
||||||
--
|
--
|
||||||
|
|
|
@ -208,17 +208,11 @@ SELECT * FROM pg_indexes WHERE tablename = 'lineitem' or tablename like 'index_t
|
||||||
--
|
--
|
||||||
-- REINDEX
|
-- REINDEX
|
||||||
--
|
--
|
||||||
SET citus.log_remote_commands to on;
|
|
||||||
SET client_min_messages = LOG;
|
|
||||||
REINDEX INDEX lineitem_orderkey_index;
|
REINDEX INDEX lineitem_orderkey_index;
|
||||||
ERROR: REINDEX is not implemented for distributed relations
|
|
||||||
REINDEX TABLE lineitem;
|
REINDEX TABLE lineitem;
|
||||||
ERROR: REINDEX is not implemented for distributed relations
|
|
||||||
REINDEX SCHEMA public;
|
REINDEX SCHEMA public;
|
||||||
REINDEX DATABASE regression;
|
REINDEX DATABASE regression;
|
||||||
REINDEX SYSTEM regression;
|
REINDEX SYSTEM regression;
|
||||||
SET citus.log_remote_commands to off;
|
|
||||||
RESET client_min_messages;
|
|
||||||
--
|
--
|
||||||
-- DROP INDEX
|
-- DROP INDEX
|
||||||
--
|
--
|
||||||
|
|
|
@ -117,15 +117,11 @@ SELECT * FROM pg_indexes WHERE tablename = 'lineitem' or tablename like 'index_t
|
||||||
-- REINDEX
|
-- REINDEX
|
||||||
--
|
--
|
||||||
|
|
||||||
SET citus.log_remote_commands to on;
|
|
||||||
SET client_min_messages = LOG;
|
|
||||||
REINDEX INDEX lineitem_orderkey_index;
|
REINDEX INDEX lineitem_orderkey_index;
|
||||||
REINDEX TABLE lineitem;
|
REINDEX TABLE lineitem;
|
||||||
REINDEX SCHEMA public;
|
REINDEX SCHEMA public;
|
||||||
REINDEX DATABASE regression;
|
REINDEX DATABASE regression;
|
||||||
REINDEX SYSTEM regression;
|
REINDEX SYSTEM regression;
|
||||||
SET citus.log_remote_commands to off;
|
|
||||||
RESET client_min_messages;
|
|
||||||
|
|
||||||
--
|
--
|
||||||
-- DROP INDEX
|
-- DROP INDEX
|
||||||
|
|
Loading…
Reference in New Issue