Merge pull request #5052 from citusdata/columnar-index

Merge columnar metapage changes and basic index support
pull/5027/head^2
Onur Tirtir 2021-06-17 14:55:40 +03:00 committed by GitHub
commit b0ca823b4d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
60 changed files with 3522 additions and 559 deletions

View File

@ -27,6 +27,7 @@
#include "columnar/columnar_customscan.h"
#include "columnar/columnar_metadata.h"
#include "columnar/columnar_tableam.h"
#include "distributed/listutils.h"
typedef struct ColumnarScanPath
{
@ -50,8 +51,13 @@ typedef struct ColumnarScanState
} ColumnarScanState;
typedef bool (*PathPredicate)(Path *path);
static void ColumnarSetRelPathlistHook(PlannerInfo *root, RelOptInfo *rel, Index rti,
RangeTblEntry *rte);
static void RemovePathsByPredicate(RelOptInfo *rel, PathPredicate removePathPredicate);
static bool IsNotIndexPath(Path *path);
static Path * CreateColumnarScanPath(PlannerInfo *root, RelOptInfo *rel,
RangeTblEntry *rte);
static Cost ColumnarScanCost(RangeTblEntry *rte);
@ -137,18 +143,6 @@ columnar_customscan_init()
}
static void
clear_paths(RelOptInfo *rel)
{
rel->pathlist = NIL;
rel->partial_pathlist = NIL;
rel->cheapest_startup_path = NULL;
rel->cheapest_total_path = NULL;
rel->cheapest_unique_path = NULL;
rel->cheapest_parameterized_paths = NIL;
}
static void
ColumnarSetRelPathlistHook(PlannerInfo *root, RelOptInfo *rel, Index rti,
RangeTblEntry *rte)
@ -188,8 +182,13 @@ ColumnarSetRelPathlistHook(PlannerInfo *root, RelOptInfo *rel, Index rti,
ereport(DEBUG1, (errmsg("pathlist hook for columnar table am")));
/* we propose a new path that will be the only path for scanning this relation */
clear_paths(rel);
/*
* TODO: Since we don't have a proper costing model for
* ColumnarCustomScan, we remove other paths to force postgres
* using ColumnarCustomScan. Note that we still keep index paths
* since they still might be useful.
*/
RemovePathsByPredicate(rel, IsNotIndexPath);
add_path(rel, customPath);
}
}
@ -197,6 +196,38 @@ ColumnarSetRelPathlistHook(PlannerInfo *root, RelOptInfo *rel, Index rti,
}
/*
* RemovePathsByPredicate removes the paths that removePathPredicate
* evaluates to true from pathlist of given rel.
*/
static void
RemovePathsByPredicate(RelOptInfo *rel, PathPredicate removePathPredicate)
{
List *filteredPathList = NIL;
Path *path = NULL;
foreach_ptr(path, rel->pathlist)
{
if (!removePathPredicate(path))
{
filteredPathList = lappend(filteredPathList, path);
}
}
rel->pathlist = filteredPathList;
}
/*
* IsNotIndexPath returns true if given path is not an IndexPath.
*/
static bool
IsNotIndexPath(Path *path)
{
return !IsA(path, IndexPath);
}
static Path *
CreateColumnarScanPath(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
{

View File

@ -12,6 +12,7 @@
#include "pg_config.h"
#include "access/nbtree.h"
#include "access/table.h"
#include "catalog/pg_am.h"
#include "catalog/pg_type.h"
#include "distributed/pg_version_constants.h"
@ -25,11 +26,13 @@
#include "utils/tuplestore.h"
#include "columnar/columnar.h"
#include "columnar/columnar_storage.h"
#include "columnar/columnar_version_compat.h"
static void MemoryContextTotals(MemoryContext context, MemoryContextCounters *counters);
PG_FUNCTION_INFO_V1(columnar_store_memory_stats);
PG_FUNCTION_INFO_V1(columnar_storage_info);
/*
@ -72,6 +75,74 @@ columnar_store_memory_stats(PG_FUNCTION_ARGS)
}
/*
* columnar_storage_info - UDF to return internal storage info for a columnar relation.
*
* DDL:
* CREATE OR REPLACE FUNCTION columnar_storage_info(
* rel regclass,
* version_major OUT int4,
* version_minor OUT int4,
* storage_id OUT int8,
* reserved_stripe_id OUT int8,
* reserved_row_number OUT int8,
* reserved_offset OUT int8)
* STRICT
* LANGUAGE c AS 'MODULE_PATHNAME', 'columnar_storage_info';
*/
Datum
columnar_storage_info(PG_FUNCTION_ARGS)
{
#define STORAGE_INFO_NATTS 6
Oid relid = PG_GETARG_OID(0);
TupleDesc tupdesc;
/* Build a tuple descriptor for our result type */
if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE)
{
elog(ERROR, "return type must be a row type");
}
if (tupdesc->natts != STORAGE_INFO_NATTS)
{
elog(ERROR, "return type must have %d columns", STORAGE_INFO_NATTS);
}
Relation rel = table_open(relid, AccessShareLock);
if (!IsColumnarTableAmTable(relid))
{
ereport(ERROR, (errmsg("table \"%s\" is not a columnar table",
RelationGetRelationName(rel))));
}
RelationOpenSmgr(rel);
Datum values[STORAGE_INFO_NATTS] = { 0 };
bool nulls[STORAGE_INFO_NATTS] = { 0 };
/*
* Pass force = true so that we can inspect metapages that are not the
* current version.
*
* NB: ensure the order and number of attributes correspond to DDL
* declaration.
*/
values[0] = Int32GetDatum(ColumnarStorageGetVersionMajor(rel, true));
values[1] = Int32GetDatum(ColumnarStorageGetVersionMinor(rel, true));
values[2] = Int64GetDatum(ColumnarStorageGetStorageId(rel, true));
values[3] = Int64GetDatum(ColumnarStorageGetReservedStripeId(rel, true));
values[4] = Int64GetDatum(ColumnarStorageGetReservedRowNumber(rel, true));
values[5] = Int64GetDatum(ColumnarStorageGetReservedOffset(rel, true));
/* release lock */
table_close(rel, AccessShareLock);
HeapTuple tuple = heap_form_tuple(tupdesc, values, nulls);
PG_RETURN_DATUM(HeapTupleGetDatum(tuple));
}
/*
* MemoryContextTotals adds stats of the given memory context and its
* subtree to the given counters.

View File

@ -1,8 +1,19 @@
/*-------------------------------------------------------------------------
*
* columnar_metadata_tables.c
* columnar_metadata.c
*
* Copyright (c), Citus Data, Inc.
* Copyright (c) Citus Data, Inc.
*
* Manages metadata for columnar relations in separate, shared metadata tables
* in the "columnar" schema.
*
* * holds basic stripe information including data size and row counts
* * holds basic chunk and chunk group information like data offsets and
* min/max values (used for Chunk Group Filtering)
* * useful for fast VACUUM operations (e.g. reporting with VACUUM VERBOSE)
* * useful for stats/costing
* * maps logical row numbers to stripe IDs
* * TODO: visibility information
*
*-------------------------------------------------------------------------
*/
@ -14,7 +25,9 @@
#include "citus_version.h"
#include "columnar/columnar.h"
#include "columnar/columnar_storage.h"
#include "columnar/columnar_version_compat.h"
#include "distributed/listutils.h"
#include <sys/stat.h>
#include "access/heapam.h"
@ -30,7 +43,6 @@
#include "commands/sequence.h"
#include "commands/trigger.h"
#include "distributed/metadata_cache.h"
#include "distributed/resource_lock.h"
#include "executor/executor.h"
#include "executor/spi.h"
#include "miscadmin.h"
@ -48,28 +60,6 @@
#include "utils/relfilenodemap.h"
/*
* Content of the first page in main fork, which stores metadata at file
* level.
*/
typedef struct ColumnarMetapage
{
/*
* Store version of file format used, so we can detect files from
* previous versions if we change file format.
*/
int versionMajor;
int versionMinor;
/*
* Each of the metadata table rows are identified by a storageId.
* We store it also in the main fork so we can link metadata rows
* with data files.
*/
uint64 storageId;
} ColumnarMetapage;
typedef struct
{
Relation rel;
@ -80,14 +70,14 @@ static void InsertStripeMetadataRow(uint64 storageId, StripeMetadata *stripe);
static void GetHighestUsedAddressAndId(uint64 storageId,
uint64 *highestUsedAddress,
uint64 *highestUsedId);
static void LockForStripeReservation(Relation rel, LOCKMODE mode);
static void UnlockForStripeReservation(Relation rel, LOCKMODE mode);
static List * ReadDataFileStripeList(uint64 storageId, Snapshot snapshot);
static StripeMetadata * BuildStripeMetadata(Datum *datumArray);
static uint32 * ReadChunkGroupRowCounts(uint64 storageId, uint64 stripe, uint32
chunkGroupCount);
static Oid ColumnarStorageIdSequenceRelationId(void);
static Oid ColumnarStripeRelationId(void);
static Oid ColumnarStripeIndexRelationId(void);
static Oid ColumnarStripePKeyIndexRelationId(void);
static Oid ColumnarStripeFirstRowNumberIndexRelationId(void);
static Oid ColumnarOptionsRelationId(void);
static Oid ColumnarOptionsIndexRegclass(void);
static Oid ColumnarChunkRelationId(void);
@ -95,6 +85,8 @@ static Oid ColumnarChunkGroupRelationId(void);
static Oid ColumnarChunkIndexRelationId(void);
static Oid ColumnarChunkGroupIndexRelationId(void);
static Oid ColumnarNamespaceId(void);
static uint64 LookupStorageId(RelFileNode relfilenode);
static uint64 GetHighestUsedFirstRowNumber(uint64 storageId);
static void DeleteStorageFromColumnarMetadataTable(Oid metadataTableId,
AttrNumber storageIdAtrrNumber,
Oid storageIdIndexId,
@ -107,8 +99,6 @@ static void FinishModifyRelation(ModifyState *state);
static EState * create_estate_for_relation(Relation rel);
static bytea * DatumToBytea(Datum value, Form_pg_attribute attrForm);
static Datum ByteaToDatum(bytea *bytes, Form_pg_attribute attrForm);
static ColumnarMetapage * InitMetapage(Relation relation);
static ColumnarMetapage * ReadMetapage(RelFileNode relfilenode, bool missingOk);
static bool WriteColumnarOptions(Oid regclass, ColumnarOptions *options, bool overwrite);
PG_FUNCTION_INFO_V1(columnar_relation_storageid);
@ -140,7 +130,7 @@ typedef FormData_columnar_options *Form_columnar_options;
/* constants for columnar.stripe */
#define Natts_columnar_stripe 8
#define Natts_columnar_stripe 9
#define Anum_columnar_stripe_storageid 1
#define Anum_columnar_stripe_stripe 2
#define Anum_columnar_stripe_file_offset 3
@ -149,6 +139,7 @@ typedef FormData_columnar_options *Form_columnar_options;
#define Anum_columnar_stripe_chunk_row_count 6
#define Anum_columnar_stripe_row_count 7
#define Anum_columnar_stripe_chunk_count 8
#define Anum_columnar_stripe_first_row_number 9
/* constants for columnar.chunk_group */
#define Natts_columnar_chunkgroup 4
@ -423,7 +414,7 @@ SaveStripeSkipList(RelFileNode relfilenode, uint64 stripe, StripeSkipList *chunk
uint32 chunkIndex = 0;
uint32 columnCount = chunkList->columnCount;
ColumnarMetapage *metapage = ReadMetapage(relfilenode, false);
uint64 storageId = LookupStorageId(relfilenode);
Oid columnarChunkOid = ColumnarChunkRelationId();
Relation columnarChunk = table_open(columnarChunkOid, RowExclusiveLock);
ModifyState *modifyState = StartModifyRelation(columnarChunk);
@ -436,7 +427,7 @@ SaveStripeSkipList(RelFileNode relfilenode, uint64 stripe, StripeSkipList *chunk
&chunkList->chunkSkipNodeArray[columnIndex][chunkIndex];
Datum values[Natts_columnar_chunk] = {
UInt64GetDatum(metapage->storageId),
UInt64GetDatum(storageId),
Int64GetDatum(stripe),
Int32GetDatum(columnIndex + 1),
Int32GetDatum(chunkIndex),
@ -487,7 +478,7 @@ void
SaveChunkGroups(RelFileNode relfilenode, uint64 stripe,
List *chunkGroupRowCounts)
{
ColumnarMetapage *metapage = ReadMetapage(relfilenode, false);
uint64 storageId = LookupStorageId(relfilenode);
Oid columnarChunkGroupOid = ColumnarChunkGroupRelationId();
Relation columnarChunkGroup = table_open(columnarChunkGroupOid, RowExclusiveLock);
ModifyState *modifyState = StartModifyRelation(columnarChunkGroup);
@ -499,7 +490,7 @@ SaveChunkGroups(RelFileNode relfilenode, uint64 stripe,
{
int64 rowCount = lfirst_int(lc);
Datum values[Natts_columnar_chunkgroup] = {
UInt64GetDatum(metapage->storageId),
UInt64GetDatum(storageId),
Int64GetDatum(stripe),
Int32GetDatum(chunkId),
Int64GetDatum(rowCount)
@ -530,14 +521,14 @@ ReadStripeSkipList(RelFileNode relfilenode, uint64 stripe, TupleDesc tupleDescri
uint32 columnCount = tupleDescriptor->natts;
ScanKeyData scanKey[2];
ColumnarMetapage *metapage = ReadMetapage(relfilenode, false);
uint64 storageId = LookupStorageId(relfilenode);
Oid columnarChunkOid = ColumnarChunkRelationId();
Relation columnarChunk = table_open(columnarChunkOid, AccessShareLock);
Relation index = index_open(ColumnarChunkIndexRelationId(), AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_columnar_chunk_storageid,
BTEqualStrategyNumber, F_OIDEQ, UInt64GetDatum(metapage->storageId));
BTEqualStrategyNumber, F_OIDEQ, UInt64GetDatum(storageId));
ScanKeyInit(&scanKey[1], Anum_columnar_chunk_stripe,
BTEqualStrategyNumber, F_OIDEQ, Int32GetDatum(stripe));
@ -624,12 +615,99 @@ ReadStripeSkipList(RelFileNode relfilenode, uint64 stripe, TupleDesc tupleDescri
table_close(columnarChunk, AccessShareLock);
chunkList->chunkGroupRowCounts =
ReadChunkGroupRowCounts(metapage->storageId, stripe, chunkCount);
ReadChunkGroupRowCounts(storageId, stripe, chunkCount);
return chunkList;
}
/*
* FindStripeByRowNumber returns StripeMetadata for the stripe that has the
* row with rowNumber by doing backward index scan on
* stripe_first_row_number_idx. If no such row exists, then returns NULL.
*/
StripeMetadata *
FindStripeByRowNumber(Relation relation, uint64 rowNumber, Snapshot snapshot)
{
StripeMetadata *foundStripeMetadata = NULL;
uint64 storageId = ColumnarStorageGetStorageId(relation, false);
ScanKeyData scanKey[2];
ScanKeyInit(&scanKey[0], Anum_columnar_stripe_storageid,
BTEqualStrategyNumber, F_OIDEQ, Int32GetDatum(storageId));
ScanKeyInit(&scanKey[1], Anum_columnar_stripe_first_row_number,
BTLessEqualStrategyNumber, F_INT8LE, UInt64GetDatum(rowNumber));
Relation columnarStripes = table_open(ColumnarStripeRelationId(), AccessShareLock);
Relation index = index_open(ColumnarStripeFirstRowNumberIndexRelationId(),
AccessShareLock);
SysScanDesc scanDescriptor = systable_beginscan_ordered(columnarStripes, index,
snapshot, 2,
scanKey);
HeapTuple heapTuple = systable_getnext_ordered(scanDescriptor, BackwardScanDirection);
if (HeapTupleIsValid(heapTuple))
{
TupleDesc tupleDescriptor = RelationGetDescr(columnarStripes);
Datum datumArray[Natts_columnar_stripe];
bool isNullArray[Natts_columnar_stripe];
heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray);
StripeMetadata *stripeMetadata = BuildStripeMetadata(datumArray);
if (rowNumber < stripeMetadata->firstRowNumber + stripeMetadata->rowCount)
{
foundStripeMetadata = stripeMetadata;
}
}
systable_endscan_ordered(scanDescriptor);
index_close(index, AccessShareLock);
table_close(columnarStripes, AccessShareLock);
return foundStripeMetadata;
}
/*
* FindStripeWithHighestRowNumber returns StripeMetadata for the stripe that
* has the row with highest rowNumber by doing backward index scan on
* stripe_first_row_number_idx. If given relation is empty, then returns NULL.
*/
StripeMetadata *
FindStripeWithHighestRowNumber(Relation relation, Snapshot snapshot)
{
StripeMetadata *stripeWithHighestRowNumber = NULL;
uint64 storageId = ColumnarStorageGetStorageId(relation, false);
ScanKeyData scanKey[1];
ScanKeyInit(&scanKey[0], Anum_columnar_stripe_storageid,
BTEqualStrategyNumber, F_OIDEQ, Int32GetDatum(storageId));
Relation columnarStripes = table_open(ColumnarStripeRelationId(), AccessShareLock);
Relation index = index_open(ColumnarStripeFirstRowNumberIndexRelationId(),
AccessShareLock);
SysScanDesc scanDescriptor = systable_beginscan_ordered(columnarStripes, index,
snapshot, 1, scanKey);
HeapTuple heapTuple = systable_getnext_ordered(scanDescriptor, BackwardScanDirection);
if (HeapTupleIsValid(heapTuple))
{
TupleDesc tupleDescriptor = RelationGetDescr(columnarStripes);
Datum datumArray[Natts_columnar_stripe];
bool isNullArray[Natts_columnar_stripe];
heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray);
stripeWithHighestRowNumber = BuildStripeMetadata(datumArray);
}
systable_endscan_ordered(scanDescriptor);
index_close(index, AccessShareLock);
table_close(columnarStripes, AccessShareLock);
return stripeWithHighestRowNumber;
}
/*
* ReadChunkGroupRowCounts returns an array of row counts of chunk groups for the
* given stripe.
@ -704,7 +782,8 @@ InsertStripeMetadataRow(uint64 storageId, StripeMetadata *stripe)
Int32GetDatum(stripe->columnCount),
Int32GetDatum(stripe->chunkGroupRowCount),
Int64GetDatum(stripe->rowCount),
Int32GetDatum(stripe->chunkCount)
Int32GetDatum(stripe->chunkCount),
UInt64GetDatum(stripe->firstRowNumber)
};
Oid columnarStripesOid = ColumnarStripeRelationId();
@ -729,15 +808,9 @@ InsertStripeMetadataRow(uint64 storageId, StripeMetadata *stripe)
List *
StripesForRelfilenode(RelFileNode relfilenode)
{
ColumnarMetapage *metapage = ReadMetapage(relfilenode, true);
if (metapage == NULL)
{
/* empty relation */
return NIL;
}
uint64 storageId = LookupStorageId(relfilenode);
return ReadDataFileStripeList(metapage->storageId, GetTransactionSnapshot());
return ReadDataFileStripeList(storageId, GetTransactionSnapshot());
}
@ -752,17 +825,11 @@ StripesForRelfilenode(RelFileNode relfilenode)
uint64
GetHighestUsedAddress(RelFileNode relfilenode)
{
uint64 storageId = LookupStorageId(relfilenode);
uint64 highestUsedAddress = 0;
uint64 highestUsedId = 0;
ColumnarMetapage *metapage = ReadMetapage(relfilenode, true);
/* empty data file? */
if (metapage == NULL)
{
return 0;
}
GetHighestUsedAddressAndId(metapage->storageId, &highestUsedAddress, &highestUsedId);
GetHighestUsedAddressAndId(storageId, &highestUsedAddress, &highestUsedId);
return highestUsedAddress;
}
@ -799,35 +866,6 @@ GetHighestUsedAddressAndId(uint64 storageId,
}
/*
* LockForStripeReservation acquires a lock for stripe reservation.
*/
static void
LockForStripeReservation(Relation rel, LOCKMODE mode)
{
/*
* We use an advisory lock here so we can easily detect these kind of
* locks in IsProcessWaitingForSafeOperations() and don't include them
* in the lock graph.
*/
LOCKTAG tag;
SET_LOCKTAG_COLUMNAR_STRIPE_RESERVATION(tag, rel);
LockAcquire(&tag, mode, false, false);
}
/*
* UnlockForStripeReservation releases the stripe reservation lock.
*/
static void
UnlockForStripeReservation(Relation rel, LOCKMODE mode)
{
LOCKTAG tag;
SET_LOCKTAG_COLUMNAR_STRIPE_RESERVATION(tag, rel);
LockRelease(&tag, mode, false);
}
/*
* ReserveStripe reserves and stripe of given size for the given relation,
* and inserts it into columnar.stripe. It is guaranteed that concurrent
@ -836,50 +874,15 @@ UnlockForStripeReservation(Relation rel, LOCKMODE mode)
StripeMetadata
ReserveStripe(Relation rel, uint64 sizeBytes,
uint64 rowCount, uint64 columnCount,
uint64 chunkCount, uint64 chunkGroupRowCount)
uint64 chunkCount, uint64 chunkGroupRowCount,
uint64 stripeFirstRowNumber)
{
StripeMetadata stripe = { 0 };
uint64 currLogicalHigh = 0;
uint64 highestId = 0;
/*
* We take ExclusiveLock here, so two space reservations conflict.
*/
LOCKMODE lockMode = ExclusiveLock;
LockForStripeReservation(rel, lockMode);
uint64 storageId = ColumnarStorageGetStorageId(rel, false);
RelFileNode relfilenode = rel->rd_node;
/*
* If this is the first stripe for this relation, initialize the
* metapage, otherwise use the previously initialized metapage.
*/
ColumnarMetapage *metapage = ReadMetapage(relfilenode, true);
if (metapage == NULL)
{
metapage = InitMetapage(rel);
}
GetHighestUsedAddressAndId(metapage->storageId, &currLogicalHigh, &highestId);
SmgrAddr currSmgrHigh = logical_to_smgr(currLogicalHigh);
SmgrAddr resSmgrStart = next_block_start(currSmgrHigh);
uint64 resLogicalStart = smgr_to_logical(resSmgrStart);
uint64 resLogicalEnd = resLogicalStart + sizeBytes - 1;
SmgrAddr resSmgrEnd = logical_to_smgr(resLogicalEnd);
RelationOpenSmgr(rel);
uint64 nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
while (resSmgrEnd.blockno >= nblocks)
{
Buffer newBuffer = ReadBuffer(rel, P_NEW);
ReleaseBuffer(newBuffer);
nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
}
RelationCloseSmgr(rel);
uint64 stripeId = ColumnarStorageReserveStripe(rel);
uint64 resLogicalStart = ColumnarStorageReserveData(rel, sizeBytes);
stripe.fileOffset = resLogicalStart;
stripe.dataLength = sizeBytes;
@ -887,11 +890,10 @@ ReserveStripe(Relation rel, uint64 sizeBytes,
stripe.chunkGroupRowCount = chunkGroupRowCount;
stripe.columnCount = columnCount;
stripe.rowCount = rowCount;
stripe.id = highestId + 1;
stripe.id = stripeId;
stripe.firstRowNumber = stripeFirstRowNumber;
InsertStripeMetadataRow(metapage->storageId, &stripe);
UnlockForStripeReservation(rel, lockMode);
InsertStripeMetadataRow(storageId, &stripe);
return stripe;
}
@ -914,7 +916,8 @@ ReadDataFileStripeList(uint64 storageId, Snapshot snapshot)
Oid columnarStripesOid = ColumnarStripeRelationId();
Relation columnarStripes = table_open(columnarStripesOid, AccessShareLock);
Relation index = index_open(ColumnarStripeIndexRelationId(), AccessShareLock);
Relation index = index_open(ColumnarStripeFirstRowNumberIndexRelationId(),
AccessShareLock);
TupleDesc tupleDescriptor = RelationGetDescr(columnarStripes);
SysScanDesc scanDescriptor = systable_beginscan_ordered(columnarStripes, index,
@ -927,22 +930,7 @@ ReadDataFileStripeList(uint64 storageId, Snapshot snapshot)
bool isNullArray[Natts_columnar_stripe];
heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray);
StripeMetadata *stripeMetadata = palloc0(sizeof(StripeMetadata));
stripeMetadata->id = DatumGetInt64(datumArray[Anum_columnar_stripe_stripe - 1]);
stripeMetadata->fileOffset = DatumGetInt64(
datumArray[Anum_columnar_stripe_file_offset - 1]);
stripeMetadata->dataLength = DatumGetInt64(
datumArray[Anum_columnar_stripe_data_length - 1]);
stripeMetadata->columnCount = DatumGetInt32(
datumArray[Anum_columnar_stripe_column_count - 1]);
stripeMetadata->chunkCount = DatumGetInt32(
datumArray[Anum_columnar_stripe_chunk_count - 1]);
stripeMetadata->chunkGroupRowCount = DatumGetInt32(
datumArray[Anum_columnar_stripe_chunk_row_count - 1]);
stripeMetadata->rowCount = DatumGetInt64(
datumArray[Anum_columnar_stripe_row_count - 1]);
StripeMetadata *stripeMetadata = BuildStripeMetadata(datumArray);
stripeMetadataList = lappend(stripeMetadataList, stripeMetadata);
}
@ -954,6 +942,32 @@ ReadDataFileStripeList(uint64 storageId, Snapshot snapshot)
}
/*
* BuildStripeMetadata builds a StripeMetadata object from given datumArray.
*/
static StripeMetadata *
BuildStripeMetadata(Datum *datumArray)
{
StripeMetadata *stripeMetadata = palloc0(sizeof(StripeMetadata));
stripeMetadata->id = DatumGetInt64(datumArray[Anum_columnar_stripe_stripe - 1]);
stripeMetadata->fileOffset = DatumGetInt64(
datumArray[Anum_columnar_stripe_file_offset - 1]);
stripeMetadata->dataLength = DatumGetInt64(
datumArray[Anum_columnar_stripe_data_length - 1]);
stripeMetadata->columnCount = DatumGetInt32(
datumArray[Anum_columnar_stripe_column_count - 1]);
stripeMetadata->chunkCount = DatumGetInt32(
datumArray[Anum_columnar_stripe_chunk_count - 1]);
stripeMetadata->chunkGroupRowCount = DatumGetInt32(
datumArray[Anum_columnar_stripe_chunk_row_count - 1]);
stripeMetadata->rowCount = DatumGetInt64(
datumArray[Anum_columnar_stripe_row_count - 1]);
stripeMetadata->firstRowNumber = DatumGetUInt64(
datumArray[Anum_columnar_stripe_first_row_number - 1]);
return stripeMetadata;
}
/*
* DeleteMetadataRows removes the rows with given relfilenode from columnar
* metadata tables.
@ -970,28 +984,20 @@ DeleteMetadataRows(RelFileNode relfilenode)
return;
}
ColumnarMetapage *metapage = ReadMetapage(relfilenode, true);
if (metapage == NULL)
{
/*
* No data has been written to this storage yet, so there is no
* associated metadata yet.
*/
return;
}
uint64 storageId = LookupStorageId(relfilenode);
DeleteStorageFromColumnarMetadataTable(ColumnarStripeRelationId(),
Anum_columnar_stripe_storageid,
ColumnarStripeIndexRelationId(),
metapage->storageId);
ColumnarStripePKeyIndexRelationId(),
storageId);
DeleteStorageFromColumnarMetadataTable(ColumnarChunkGroupRelationId(),
Anum_columnar_chunkgroup_storageid,
ColumnarChunkGroupIndexRelationId(),
metapage->storageId);
storageId);
DeleteStorageFromColumnarMetadataTable(ColumnarChunkRelationId(),
Anum_columnar_chunk_storageid,
ColumnarChunkIndexRelationId(),
metapage->storageId);
storageId);
}
@ -1226,16 +1232,28 @@ ColumnarStripeRelationId(void)
/*
* ColumnarStripeIndexRelationId returns relation id of columnar.stripe_pkey.
* ColumnarStripePKeyIndexRelationId returns relation id of columnar.stripe_pkey.
* TODO: should we cache this similar to citus?
*/
static Oid
ColumnarStripeIndexRelationId(void)
ColumnarStripePKeyIndexRelationId(void)
{
return get_relname_relid("stripe_pkey", ColumnarNamespaceId());
}
/*
* ColumnarStripeFirstRowNumberIndexRelationId returns relation id of
* columnar.stripe_first_row_number_idx.
* TODO: should we cache this similar to citus?
*/
static Oid
ColumnarStripeFirstRowNumberIndexRelationId(void)
{
return get_relname_relid("stripe_first_row_number_idx", ColumnarNamespaceId());
}
/*
* ColumnarOptionsRelationId returns relation id of columnar.options.
*/
@ -1312,75 +1330,31 @@ ColumnarNamespaceId(void)
/*
* ReadMetapage reads metapage for the given relfilenode. It returns
* LookupStorageId reads storage metapage to find the storage ID for the given relfilenode. It returns
* false if the relation doesn't have a meta page yet.
*/
static ColumnarMetapage *
ReadMetapage(RelFileNode relfilenode, bool missingOk)
static uint64
LookupStorageId(RelFileNode relfilenode)
{
StringInfo metapageBuffer = NULL;
Oid relationId = RelidByRelfilenode(relfilenode.spcNode,
relfilenode.relNode);
if (OidIsValid(relationId))
{
Relation relation = relation_open(relationId, NoLock);
RelationOpenSmgr(relation);
int nblocks = smgrnblocks(relation->rd_smgr, MAIN_FORKNUM);
RelationCloseSmgr(relation);
Relation relation = relation_open(relationId, AccessShareLock);
uint64 storageId = ColumnarStorageGetStorageId(relation, false);
table_close(relation, AccessShareLock);
if (nblocks != 0)
{
metapageBuffer = ReadFromSmgr(relation, 0, sizeof(ColumnarMetapage));
}
relation_close(relation, NoLock);
}
if (metapageBuffer == NULL)
{
if (!missingOk)
{
elog(ERROR, "columnar metapage was not found");
}
return NULL;
}
ColumnarMetapage *metapage = palloc0(sizeof(ColumnarMetapage));
memcpy_s((void *) metapage, sizeof(ColumnarMetapage),
metapageBuffer->data, sizeof(ColumnarMetapage));
return metapage;
return storageId;
}
/*
* InitMetapage initializes metapage for the given relation.
* ColumnarMetadataNewStorageId - create a new, unique storage id and return
* it.
*/
static ColumnarMetapage *
InitMetapage(Relation relation)
uint64
ColumnarMetadataNewStorageId()
{
/*
* If we init metapage during upgrade, we might override the
* pre-upgrade storage id which will render pre-upgrade data
* invisible.
*/
Assert(!IsBinaryUpgrade);
ColumnarMetapage *metapage = palloc0(sizeof(ColumnarMetapage));
metapage->storageId = nextval_internal(ColumnarStorageIdSequenceRelationId(), false);
metapage->versionMajor = COLUMNAR_VERSION_MAJOR;
metapage->versionMinor = COLUMNAR_VERSION_MINOR;
/* create the first block */
Buffer newBuffer = ReadBuffer(relation, P_NEW);
ReleaseBuffer(newBuffer);
Assert(sizeof(ColumnarMetapage) <= BLCKSZ - SizeOfPageHeaderData);
WriteToSmgr(relation, 0, (char *) metapage, sizeof(ColumnarMetapage));
return metapage;
return nextval_internal(ColumnarStorageIdSequenceRelationId(), false);
}
@ -1391,20 +1365,85 @@ InitMetapage(Relation relation)
Datum
columnar_relation_storageid(PG_FUNCTION_ARGS)
{
uint64 storageId = -1;
Oid relationId = PG_GETARG_OID(0);
Relation relation = relation_open(relationId, AccessShareLock);
if (IsColumnarTableAmTable(relationId))
if (!IsColumnarTableAmTable(relationId))
{
ColumnarMetapage *metadata = ReadMetapage(relation->rd_node, true);
if (metadata != NULL)
{
storageId = metadata->storageId;
}
elog(ERROR, "relation \"%s\" is not a columnar table",
RelationGetRelationName(relation));
}
uint64 storageId = ColumnarStorageGetStorageId(relation, false);
relation_close(relation, AccessShareLock);
PG_RETURN_INT64(storageId);
}
/*
* ColumnarStorageUpdateIfNeeded - upgrade columnar storage to the current version by
* using information from the metadata tables.
*/
void
ColumnarStorageUpdateIfNeeded(Relation rel, bool isUpgrade)
{
if (ColumnarStorageIsCurrent(rel))
{
return;
}
RelationOpenSmgr(rel);
BlockNumber nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
if (nblocks < 2)
{
ColumnarStorageInit(rel->rd_smgr, ColumnarMetadataNewStorageId());
return;
}
uint64 storageId = ColumnarStorageGetStorageId(rel, true);
uint64 highestId;
uint64 highestOffset;
GetHighestUsedAddressAndId(storageId, &highestOffset, &highestId);
uint64 reservedStripeId = highestId + 1;
uint64 reservedOffset = highestOffset + 1;
uint64 reservedRowNumber = GetHighestUsedFirstRowNumber(storageId) + 1;
ColumnarStorageUpdateCurrent(rel, isUpgrade, reservedStripeId,
reservedRowNumber, reservedOffset);
}
/*
* GetHighestUsedFirstRowNumber returns the highest used first_row_number
* for given storageId. Returns COLUMNAR_INVALID_ROW_NUMBER if storage with
* storageId has no stripes.
* Note that normally we would use ColumnarStorageGetReservedRowNumber
* to decide that. However, this function is designed to be used when
* building the metapage itself during upgrades.
*/
static uint64
GetHighestUsedFirstRowNumber(uint64 storageId)
{
List *stripeMetadataList = ReadDataFileStripeList(storageId,
GetTransactionSnapshot());
if (list_length(stripeMetadataList) == 0)
{
return COLUMNAR_INVALID_ROW_NUMBER;
}
/* XXX: Better to have an invalid value for StripeMetadata.rowCount too */
uint64 stripeRowCount = -1;
uint64 highestFirstRowNumber = COLUMNAR_INVALID_ROW_NUMBER;
StripeMetadata *stripeMetadata = NULL;
foreach_ptr(stripeMetadata, stripeMetadataList)
{
highestFirstRowNumber = Max(highestFirstRowNumber,
stripeMetadata->firstRowNumber);
stripeRowCount = stripeMetadata->rowCount;
}
return highestFirstRowNumber + stripeRowCount - 1;
}

View File

@ -34,6 +34,8 @@
#include "utils/rel.h"
#include "columnar/columnar.h"
#include "columnar/columnar_storage.h"
#include "columnar/columnar_tableam.h"
#include "columnar/columnar_version_compat.h"
typedef struct ChunkGroupReadState
@ -84,6 +86,14 @@ struct ColumnarReadState
/* static function declarations */
static MemoryContext CreateStripeReadMemoryContext(void);
static void ReadStripeRowByRowNumber(StripeReadState *stripeReadState,
StripeMetadata *stripeMetadata,
uint64 rowNumber, Datum *columnValues,
bool *columnNulls);
static void ReadChunkGroupRowByRowOffset(ChunkGroupReadState *chunkGroupReadState,
StripeMetadata *stripeMetadata,
uint64 stripeRowOffset, Datum *columnValues,
bool *columnNulls);
static bool StripeReadInProgress(ColumnarReadState *readState);
static bool HasUnreadStripe(ColumnarReadState *readState);
static StripeReadState * BeginStripeRead(StripeMetadata *stripeMetadata, Relation rel,
@ -194,11 +204,12 @@ CreateStripeReadMemoryContext()
/*
* ColumnarReadNextRow tries to read a row from the columnar table. On success, it sets
* column values and nulls, and returns true. If there are no more rows to read,
* the function returns false.
* column values, column nulls and rowNumber (if passed to be non-NULL), and returns true.
* If there are no more rows to read, the function returns false.
*/
bool
ColumnarReadNextRow(ColumnarReadState *readState, Datum *columnValues, bool *columnNulls)
ColumnarReadNextRow(ColumnarReadState *readState, Datum *columnValues, bool *columnNulls,
uint64 *rowNumber)
{
while (true)
{
@ -226,6 +237,14 @@ ColumnarReadNextRow(ColumnarReadState *readState, Datum *columnValues, bool *col
continue;
}
if (rowNumber)
{
StripeMetadata *stripeMetadata = list_nth(readState->stripeList,
readState->currentStripe);
*rowNumber = stripeMetadata->firstRowNumber +
readState->stripeReadState->currentRow - 1;
}
return true;
}
@ -233,6 +252,104 @@ ColumnarReadNextRow(ColumnarReadState *readState, Datum *columnValues, bool *col
}
/*
* ColumnarReadRowByRowNumber reads row with rowNumber from given relation
* into columnValues and columnNulls, and returns true. If no such row
* exists, then returns false.
*/
bool
ColumnarReadRowByRowNumber(Relation relation, uint64 rowNumber,
List *neededColumnList, Datum *columnValues,
bool *columnNulls, Snapshot snapshot)
{
StripeMetadata *stripeMetadata = FindStripeByRowNumber(relation, rowNumber, snapshot);
if (stripeMetadata == NULL)
{
/* no such row exists */
return false;
}
TupleDesc relationTupleDesc = RelationGetDescr(relation);
List *whereClauseList = NIL;
List *whereClauseVars = NIL;
MemoryContext stripeReadContext = CreateStripeReadMemoryContext();
StripeReadState *stripeReadState = BeginStripeRead(stripeMetadata,
relation,
relationTupleDesc,
neededColumnList,
whereClauseList,
whereClauseVars,
stripeReadContext);
ReadStripeRowByRowNumber(stripeReadState, stripeMetadata, rowNumber,
columnValues, columnNulls);
EndStripeRead(stripeReadState);
MemoryContextReset(stripeReadContext);
return true;
}
/*
* ReadStripeRowByRowNumber reads row with rowNumber from given
* stripeReadState into columnValues and columnNulls.
* Errors out if no such row exists in the stripe being read.
*/
static void
ReadStripeRowByRowNumber(StripeReadState *stripeReadState,
StripeMetadata *stripeMetadata,
uint64 rowNumber, Datum *columnValues,
bool *columnNulls)
{
if (rowNumber < stripeMetadata->firstRowNumber)
{
/* not expected but be on the safe side */
ereport(ERROR, (errmsg("row offset cannot be negative")));
}
/* find the exact chunk group to be read */
uint64 stripeRowOffset = rowNumber - stripeMetadata->firstRowNumber;
stripeReadState->chunkGroupIndex = stripeRowOffset /
stripeMetadata->chunkGroupRowCount;
stripeReadState->chunkGroupReadState = BeginChunkGroupRead(
stripeReadState->stripeBuffers,
stripeReadState->chunkGroupIndex,
stripeReadState->tupleDescriptor,
stripeReadState->projectedColumnList,
stripeReadState->stripeReadContext);
ReadChunkGroupRowByRowOffset(stripeReadState->chunkGroupReadState,
stripeMetadata, stripeRowOffset,
columnValues, columnNulls);
EndChunkGroupRead(stripeReadState->chunkGroupReadState);
stripeReadState->chunkGroupReadState = NULL;
}
/*
* ReadChunkGroupRowByRowOffset reads row with stripeRowOffset from given
* chunkGroupReadState into columnValues and columnNulls.
* Errors out if no such row exists in the chunk group being read.
*/
static void
ReadChunkGroupRowByRowOffset(ChunkGroupReadState *chunkGroupReadState,
StripeMetadata *stripeMetadata,
uint64 stripeRowOffset, Datum *columnValues,
bool *columnNulls)
{
/* set the exact row number to be read from given chunk roup */
chunkGroupReadState->currentRow = stripeRowOffset %
stripeMetadata->chunkGroupRowCount;
if (!ReadChunkGroupNextRow(chunkGroupReadState, columnValues, columnNulls))
{
/* not expected but be on the safe side */
ereport(ERROR, (errmsg("could not find the row in stripe")));
}
}
/*
* StripeReadInProgress returns true if we already started reading a stripe.
*/
@ -667,8 +784,12 @@ LoadColumnBuffers(Relation relation, ColumnChunkSkipNode *chunkSkipNodeArray,
{
ColumnChunkSkipNode *chunkSkipNode = &chunkSkipNodeArray[chunkIndex];
uint64 existsOffset = stripeOffset + chunkSkipNode->existsChunkOffset;
StringInfo rawExistsBuffer = ReadFromSmgr(relation, existsOffset,
chunkSkipNode->existsLength);
StringInfo rawExistsBuffer = makeStringInfo();
enlargeStringInfo(rawExistsBuffer, chunkSkipNode->existsLength);
rawExistsBuffer->len = chunkSkipNode->existsLength;
ColumnarStorageRead(relation, existsOffset, rawExistsBuffer->data,
chunkSkipNode->existsLength);
chunkBuffersArray[chunkIndex]->existsBuffer = rawExistsBuffer;
}
@ -679,8 +800,12 @@ LoadColumnBuffers(Relation relation, ColumnChunkSkipNode *chunkSkipNodeArray,
ColumnChunkSkipNode *chunkSkipNode = &chunkSkipNodeArray[chunkIndex];
CompressionType compressionType = chunkSkipNode->valueCompressionType;
uint64 valueOffset = stripeOffset + chunkSkipNode->valueChunkOffset;
StringInfo rawValueBuffer = ReadFromSmgr(relation, valueOffset,
chunkSkipNode->valueLength);
StringInfo rawValueBuffer = makeStringInfo();
enlargeStringInfo(rawValueBuffer, chunkSkipNode->valueLength);
rawValueBuffer->len = chunkSkipNode->valueLength;
ColumnarStorageRead(relation, valueOffset, rawValueBuffer->data,
chunkSkipNode->valueLength);
chunkBuffersArray[chunkIndex]->valueBuffer = rawValueBuffer;
chunkBuffersArray[chunkIndex]->valueCompressionType = compressionType;
@ -1269,30 +1394,3 @@ ColumnDefaultValue(TupleConstr *tupleConstraints, Form_pg_attribute attributeFor
"does not evaluate to constant value")));
}
}
StringInfo
ReadFromSmgr(Relation rel, uint64 offset, uint32 size)
{
StringInfo resultBuffer = makeStringInfo();
uint64 read = 0;
enlargeStringInfo(resultBuffer, size);
resultBuffer->len = size;
while (read < size)
{
SmgrAddr addr = logical_to_smgr(offset + read);
Buffer buffer = ReadBuffer(rel, addr.blockno);
Page page = BufferGetPage(buffer);
PageHeader phdr = (PageHeader) page;
uint32 to_read = Min(size - read, phdr->pd_upper - addr.offset);
memcpy_s(resultBuffer->data + read, size - read, page + addr.offset, to_read);
ReleaseBuffer(buffer);
read += to_read;
}
return resultBuffer;
}

View File

@ -0,0 +1,809 @@
/*-------------------------------------------------------------------------
*
* columnar_storage.c
*
* Copyright (c) Citus Data, Inc.
*
* Low-level storage layer for columnar.
* - Translates columnar read/write operations on logical offsets into operations on pages/blocks.
* - Emits WAL.
* - Reads/writes the columnar metapage.
* - Reserves data offsets, stripe numbers, and row offsets.
* - Truncation.
*
* Higher-level columnar operations deal with logical offsets and large
* contiguous buffers of data that need to be stored. But the buffer manager
* and WAL depend on formatted pages with headers, so these large buffers need
* to be written across many pages. This module translates the contiguous
* buffers into individual block reads/writes, and performs WAL when
* necessary.
*
* Storage layout: a metapage in block 0, followed by an empty page in block
* 1, followed by logical data starting at the first byte after the page
* header in block 2 (having logical offset ColumnarFirstLogicalOffset). (XXX:
* Block 1 is left empty for no particular reason. Reconsider?). A columnar
* table should always have at least 2 blocks.
*
* Reservation is done with a relation extension lock, and designed for
* concurrency, so the callers only need an ordinary lock on the
* relation. Initializing the metapage or truncating the relation require that
* the caller holds an AccessExclusiveLock. (XXX: New reservations of data are
* aligned onto a new page for no particular reason. Reconsider?).
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "safe_lib.h"
#include "catalog/storage.h"
#include "miscadmin.h"
#include "storage/bufmgr.h"
#include "storage/lmgr.h"
#include "columnar/columnar.h"
#include "columnar/columnar_storage.h"
/*
* Content of the first page in main fork, which stores metadata at file
* level.
*/
typedef struct ColumnarMetapage
{
/*
* Store version of file format used, so we can detect files from
* previous versions if we change file format.
*/
uint32 versionMajor;
uint32 versionMinor;
/*
* Each of the metadata table rows are identified by a storageId.
* We store it also in the main fork so we can link metadata rows
* with data files.
*/
uint64 storageId;
uint64 reservedStripeId; /* first unused stripe id */
uint64 reservedRowNumber; /* first unused row number */
uint64 reservedOffset; /* first unused byte offset */
} ColumnarMetapage;
/* represents a "physical" block+offset address */
typedef struct PhysicalAddr
{
BlockNumber blockno;
uint32 offset;
} PhysicalAddr;
#define COLUMNAR_METAPAGE_BLOCKNO 0
#define COLUMNAR_EMPTY_BLOCKNO 1
#define COLUMNAR_INVALID_STRIPE_ID 0
#define COLUMNAR_FIRST_STRIPE_ID 1
#define OLD_METAPAGE_VERSION_HINT "Use \"VACUUM\" to upgrade the columnar table format " \
"version or run \"ALTER EXTENSION citus UPDATE\"."
/*
* Map logical offsets to a physical page and offset where the data is kept.
*/
static inline PhysicalAddr
LogicalToPhysical(uint64 logicalOffset)
{
PhysicalAddr addr;
addr.blockno = logicalOffset / COLUMNAR_BYTES_PER_PAGE;
addr.offset = SizeOfPageHeaderData + (logicalOffset % COLUMNAR_BYTES_PER_PAGE);
return addr;
}
/*
* Map a physical page and offset address to a logical address.
*/
static inline uint64
PhysicalToLogical(PhysicalAddr addr)
{
return COLUMNAR_BYTES_PER_PAGE * addr.blockno + addr.offset - SizeOfPageHeaderData;
}
static void ColumnarOverwriteMetapage(Relation relation,
ColumnarMetapage columnarMetapage);
static ColumnarMetapage ColumnarMetapageRead(Relation rel, bool force);
static void ReadFromBlock(Relation rel, BlockNumber blockno, uint32 offset,
char *buf, uint32 len, bool force);
static void WriteToBlock(Relation rel, BlockNumber blockno, uint32 offset,
char *buf, uint32 len, bool clear);
static uint64 AlignReservation(uint64 prevReservation);
static bool ColumnarMetapageIsCurrent(ColumnarMetapage *metapage);
static bool ColumnarMetapageIsOlder(ColumnarMetapage *metapage);
static bool ColumnarMetapageIsNewer(ColumnarMetapage *metapage);
static void ColumnarMetapageCheckVersion(Relation rel, ColumnarMetapage *metapage);
/*
* ColumnarStorageInit - initialize a new metapage in an empty relation
* with the given storageId.
*
* Caller must hold AccessExclusiveLock on the relation.
*/
void
ColumnarStorageInit(SMgrRelation srel, uint64 storageId)
{
BlockNumber nblocks = smgrnblocks(srel, MAIN_FORKNUM);
if (nblocks > 0)
{
elog(ERROR,
"attempted to initialize metapage, but %d pages already exist",
nblocks);
}
/* create two pages */
PGAlignedBlock block;
Page page = block.data;
/* write metapage */
PageInit(page, BLCKSZ, 0);
PageHeader phdr = (PageHeader) page;
ColumnarMetapage metapage = { 0 };
metapage.storageId = storageId;
metapage.versionMajor = COLUMNAR_VERSION_MAJOR;
metapage.versionMinor = COLUMNAR_VERSION_MINOR;
metapage.reservedStripeId = COLUMNAR_FIRST_STRIPE_ID;
metapage.reservedRowNumber = COLUMNAR_FIRST_ROW_NUMBER;
metapage.reservedOffset = ColumnarFirstLogicalOffset;
memcpy_s(page + phdr->pd_lower, phdr->pd_upper - phdr->pd_lower,
(char *) &metapage, sizeof(ColumnarMetapage));
phdr->pd_lower += sizeof(ColumnarMetapage);
PageSetChecksumInplace(page, COLUMNAR_METAPAGE_BLOCKNO);
smgrwrite(srel, MAIN_FORKNUM, COLUMNAR_METAPAGE_BLOCKNO, page, true);
log_newpage(&srel->smgr_rnode.node, MAIN_FORKNUM,
COLUMNAR_METAPAGE_BLOCKNO, page, true);
/* write empty page */
PageInit(page, BLCKSZ, 0);
PageSetChecksumInplace(page, COLUMNAR_EMPTY_BLOCKNO);
smgrwrite(srel, MAIN_FORKNUM, COLUMNAR_EMPTY_BLOCKNO, page, true);
log_newpage(&srel->smgr_rnode.node, MAIN_FORKNUM,
COLUMNAR_EMPTY_BLOCKNO, page, true);
/*
* An immediate sync is required even if we xlog'd the page, because the
* write did not go through shared_buffers and therefore a concurrent
* checkpoint may have moved the redo pointer past our xlog record.
*/
smgrimmedsync(srel, MAIN_FORKNUM);
}
/*
* ColumnarStorageUpdateCurrent - update the metapage to the current
* version. No effect if the version already matches. If 'upgrade' is true,
* throw an error if metapage version is newer; if 'upgrade' is false, it's a
* downgrade, so throw an error if the metapage version is older.
*
* NB: caller must ensure that metapage already exists, which might not be the
* case on 10.0.
*/
void
ColumnarStorageUpdateCurrent(Relation rel, bool upgrade, uint64 reservedStripeId,
uint64 reservedRowNumber, uint64 reservedOffset)
{
LockRelationForExtension(rel, ExclusiveLock);
ColumnarMetapage metapage = ColumnarMetapageRead(rel, true);
if (ColumnarMetapageIsCurrent(&metapage))
{
/* nothing to do */
return;
}
if (upgrade && ColumnarMetapageIsNewer(&metapage))
{
elog(ERROR, "found newer columnar metapage while upgrading");
}
if (!upgrade && ColumnarMetapageIsOlder(&metapage))
{
elog(ERROR, "found older columnar metapage while downgrading");
}
metapage.versionMajor = COLUMNAR_VERSION_MAJOR;
metapage.versionMinor = COLUMNAR_VERSION_MINOR;
/* storageId remains the same */
metapage.reservedStripeId = reservedStripeId;
metapage.reservedRowNumber = reservedRowNumber;
metapage.reservedOffset = reservedOffset;
ColumnarOverwriteMetapage(rel, metapage);
UnlockRelationForExtension(rel, ExclusiveLock);
}
/*
* ColumnarStorageGetVersionMajor - return major version from the metapage.
*
* Throw an error if the metapage is not the current version, unless
* 'force' is true.
*/
uint64
ColumnarStorageGetVersionMajor(Relation rel, bool force)
{
ColumnarMetapage metapage = ColumnarMetapageRead(rel, force);
return metapage.versionMajor;
}
/*
* ColumnarStorageGetVersionMinor - return minor version from the metapage.
*
* Throw an error if the metapage is not the current version, unless
* 'force' is true.
*/
uint64
ColumnarStorageGetVersionMinor(Relation rel, bool force)
{
ColumnarMetapage metapage = ColumnarMetapageRead(rel, force);
return metapage.versionMinor;
}
/*
* ColumnarStorageGetStorageId - return storage ID from the metapage.
*
* Throw an error if the metapage is not the current version, unless
* 'force' is true.
*/
uint64
ColumnarStorageGetStorageId(Relation rel, bool force)
{
ColumnarMetapage metapage = ColumnarMetapageRead(rel, force);
return metapage.storageId;
}
/*
* ColumnarStorageGetReservedStripeId - return reserved stripe ID from the
* metapage.
*
* Throw an error if the metapage is not the current version, unless
* 'force' is true.
*/
uint64
ColumnarStorageGetReservedStripeId(Relation rel, bool force)
{
ColumnarMetapage metapage = ColumnarMetapageRead(rel, force);
return metapage.reservedStripeId;
}
/*
* ColumnarStorageGetReservedRowNumber - return reserved row number from the
* metapage.
*
* Throw an error if the metapage is not the current version, unless
* 'force' is true.
*/
uint64
ColumnarStorageGetReservedRowNumber(Relation rel, bool force)
{
ColumnarMetapage metapage = ColumnarMetapageRead(rel, force);
return metapage.reservedRowNumber;
}
/*
* ColumnarStorageGetReservedOffset - return reserved offset from the metapage.
*
* Throw an error if the metapage is not the current version, unless
* 'force' is true.
*/
uint64
ColumnarStorageGetReservedOffset(Relation rel, bool force)
{
ColumnarMetapage metapage = ColumnarMetapageRead(rel, force);
return metapage.reservedOffset;
}
/*
* ColumnarStorageIsCurrent - return true if metapage exists and is not
* the current version.
*/
bool
ColumnarStorageIsCurrent(Relation rel)
{
RelationOpenSmgr(rel);
BlockNumber nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
if (nblocks < 2)
{
return false;
}
ColumnarMetapage metapage = ColumnarMetapageRead(rel, true);
return ColumnarMetapageIsCurrent(&metapage);
}
/*
* ColumnarStorageReserveRowNumber returns reservedRowNumber and advances
* it for next row number reservation.
*/
uint64
ColumnarStorageReserveRowNumber(Relation rel, uint64 nrows)
{
LockRelationForExtension(rel, ExclusiveLock);
ColumnarMetapage metapage = ColumnarMetapageRead(rel, false);
uint64 firstRowNumber = metapage.reservedRowNumber;
metapage.reservedRowNumber += nrows;
ColumnarOverwriteMetapage(rel, metapage);
UnlockRelationForExtension(rel, ExclusiveLock);
return firstRowNumber;
}
/*
* ColumnarStorageReserveStripe returns stripeId and advances it for next
* stripeId reservation.
* Note that this function doesn't handle row number reservation.
* This is because, unlike stripeId reservation, we immediately reserve
* row number during writes, not when flushing stripes to disk.
* See ColumnarStorageReserveRowNumber function.
*/
uint64
ColumnarStorageReserveStripe(Relation rel)
{
LockRelationForExtension(rel, ExclusiveLock);
ColumnarMetapage metapage = ColumnarMetapageRead(rel, false);
uint64 stripeId = metapage.reservedStripeId;
metapage.reservedStripeId++;
ColumnarOverwriteMetapage(rel, metapage);
UnlockRelationForExtension(rel, ExclusiveLock);
return stripeId;
}
/*
* ColumnarStorageReserveData - reserve logical data offsets for writing.
*/
uint64
ColumnarStorageReserveData(Relation rel, uint64 amount)
{
if (amount == 0)
{
return ColumnarInvalidLogicalOffset;
}
LockRelationForExtension(rel, ExclusiveLock);
ColumnarMetapage metapage = ColumnarMetapageRead(rel, false);
uint64 alignedReservation = AlignReservation(metapage.reservedOffset);
uint64 nextReservation = alignedReservation + amount;
metapage.reservedOffset = nextReservation;
/* write new reservation */
ColumnarOverwriteMetapage(rel, metapage);
/* last used PhysicalAddr of new reservation */
PhysicalAddr final = LogicalToPhysical(nextReservation - 1);
/* extend with new pages */
RelationOpenSmgr(rel);
BlockNumber nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
while (nblocks <= final.blockno)
{
Buffer newBuffer = ReadBuffer(rel, P_NEW);
Assert(BufferGetBlockNumber(newBuffer) == nblocks);
ReleaseBuffer(newBuffer);
nblocks++;
}
UnlockRelationForExtension(rel, ExclusiveLock);
return alignedReservation;
}
/*
* ColumnarStorageRead - map the logical offset to a block and offset, then
* read the buffer from multiple blocks if necessary.
*/
void
ColumnarStorageRead(Relation rel, uint64 logicalOffset, char *data, uint32 amount)
{
/* if there's no work to do, succeed even with invalid offset */
if (amount == 0)
{
return;
}
if (!ColumnarLogicalOffsetIsValid(logicalOffset))
{
elog(ERROR,
"attempted columnar read on relation %d from invalid logical offset: "
UINT64_FORMAT,
rel->rd_id, logicalOffset);
}
uint64 read = 0;
while (read < amount)
{
PhysicalAddr addr = LogicalToPhysical(logicalOffset + read);
uint32 to_read = Min(amount - read, BLCKSZ - addr.offset);
ReadFromBlock(rel, addr.blockno, addr.offset, data + read, to_read,
false);
read += to_read;
}
}
/*
* ColumnarStorageWrite - map the logical offset to a block and offset, then
* write the buffer across multiple blocks if necessary.
*/
void
ColumnarStorageWrite(Relation rel, uint64 logicalOffset, char *data, uint32 amount)
{
/* if there's no work to do, succeed even with invalid offset */
if (amount == 0)
{
return;
}
if (!ColumnarLogicalOffsetIsValid(logicalOffset))
{
elog(ERROR,
"attempted columnar write on relation %d to invalid logical offset: "
UINT64_FORMAT,
rel->rd_id, logicalOffset);
}
uint64 written = 0;
while (written < amount)
{
PhysicalAddr addr = LogicalToPhysical(logicalOffset + written);
uint64 to_write = Min(amount - written, BLCKSZ - addr.offset);
WriteToBlock(rel, addr.blockno, addr.offset, data + written, to_write,
false);
written += to_write;
}
}
/*
* ColumnarStorageTruncate - truncate the columnar storage such that
* newDataReservation will be the first unused logical offset available. Free
* pages at the end of the relation.
*
* Caller must hold AccessExclusiveLock on the relation.
*
* Returns true if pages were truncated; false otherwise.
*/
bool
ColumnarStorageTruncate(Relation rel, uint64 newDataReservation)
{
if (!ColumnarLogicalOffsetIsValid(newDataReservation))
{
elog(ERROR,
"attempted to truncate relation %d to invalid logical offset: " UINT64_FORMAT,
rel->rd_id, newDataReservation);
}
RelationOpenSmgr(rel);
BlockNumber old_rel_pages = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
if (old_rel_pages == 0)
{
/* nothing to do */
return false;
}
LockRelationForExtension(rel, ExclusiveLock);
ColumnarMetapage metapage = ColumnarMetapageRead(rel, false);
if (metapage.reservedOffset < newDataReservation)
{
elog(ERROR,
"attempted to truncate relation %d to offset " UINT64_FORMAT \
" which is higher than existing offset " UINT64_FORMAT,
rel->rd_id, newDataReservation, metapage.reservedOffset);
}
if (metapage.reservedOffset == newDataReservation)
{
/* nothing to do */
UnlockRelationForExtension(rel, ExclusiveLock);
return false;
}
metapage.reservedOffset = newDataReservation;
/* write new reservation */
ColumnarOverwriteMetapage(rel, metapage);
UnlockRelationForExtension(rel, ExclusiveLock);
PhysicalAddr final = LogicalToPhysical(newDataReservation - 1);
BlockNumber new_rel_pages = final.blockno + 1;
Assert(new_rel_pages <= old_rel_pages);
/*
* Truncate the storage. Note that RelationTruncate() takes care of
* Write Ahead Logging.
*/
if (new_rel_pages < old_rel_pages)
{
RelationTruncate(rel, new_rel_pages);
return true;
}
return false;
}
/*
* ColumnarOverwriteMetapage writes given columnarMetapage back to metapage
* for given relation.
*/
static void
ColumnarOverwriteMetapage(Relation relation, ColumnarMetapage columnarMetapage)
{
/* clear metapage because we are overwriting */
bool clear = true;
WriteToBlock(relation, COLUMNAR_METAPAGE_BLOCKNO, SizeOfPageHeaderData,
(char *) &columnarMetapage, sizeof(ColumnarMetapage), clear);
}
/*
* ColumnarMetapageRead - read the current contents of the metapage. Error if
* it does not exist. Throw an error if the metapage is not the current
* version, unless 'force' is true.
*
* NB: it's safe to read a different version of a metapage because we
* guarantee that fields will only be added and existing fields will never be
* changed. However, it's important that we don't depend on new fields being
* set properly when we read an old metapage; an old metapage should only be
* read for the purposes of upgrading or error checking.
*/
static ColumnarMetapage
ColumnarMetapageRead(Relation rel, bool force)
{
RelationOpenSmgr(rel);
BlockNumber nblocks = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
if (nblocks == 0)
{
/*
* We only expect this to happen when upgrading citus.so. This is because,
* in current version of columnar, we immediately create the metapage
* for columnar tables, i.e right after creating the table.
* However in older versions, we were creating metapages lazily, i.e
* when ingesting data to columnar table.
*/
ereport(ERROR, (errmsg("columnar metapage for relation \"%s\" does not exist",
RelationGetRelationName(rel)),
errhint(OLD_METAPAGE_VERSION_HINT)));
}
/*
* Regardless of "force" parameter, always force read metapage block.
* We will check metapage version in ColumnarMetapageCheckVersion
* depending on "force".
*/
bool forceReadBlock = true;
ColumnarMetapage metapage;
ReadFromBlock(rel, COLUMNAR_METAPAGE_BLOCKNO, SizeOfPageHeaderData,
(char *) &metapage, sizeof(ColumnarMetapage), forceReadBlock);
if (!force)
{
ColumnarMetapageCheckVersion(rel, &metapage);
}
return metapage;
}
/*
* ReadFromBlock - read bytes from a page at the given offset. If 'force' is
* true, don't check pd_lower; useful when reading a metapage of unknown
* version.
*/
static void
ReadFromBlock(Relation rel, BlockNumber blockno, uint32 offset, char *buf,
uint32 len, bool force)
{
Buffer buffer = ReadBuffer(rel, blockno);
Page page = BufferGetPage(buffer);
PageHeader phdr = (PageHeader) page;
if (BLCKSZ < offset + len || (!force && (phdr->pd_lower < offset + len)))
{
elog(ERROR,
"attempt to read columnar data of length %d from offset %d of block %d of relation %d",
len, offset, blockno, rel->rd_id);
}
memcpy_s(buf, len, page + offset, len);
ReleaseBuffer(buffer);
}
/*
* WriteToBlock - append data to a block, initializing if necessary, and emit
* WAL. If 'clear' is true, always clear the data on the page and reinitialize
* it first, and offset must be SizeOfPageHeaderData. Otherwise, offset must
* be equal to pd_lower and pd_lower will be set to the end of the written
* data.
*/
static void
WriteToBlock(Relation rel, BlockNumber blockno, uint32 offset, char *buf,
uint32 len, bool clear)
{
Buffer buffer = ReadBuffer(rel, blockno);
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
Page page = BufferGetPage(buffer);
PageHeader phdr = (PageHeader) page;
if (PageIsNew(page) || clear)
{
PageInit(page, BLCKSZ, 0);
}
if (phdr->pd_lower != offset || phdr->pd_upper - offset < len)
{
elog(ERROR,
"attempt to write columnar data of length %d to offset %d of block %d of relation %d",
len, offset, blockno, rel->rd_id);
}
START_CRIT_SECTION();
memcpy_s(page + phdr->pd_lower, phdr->pd_upper - phdr->pd_lower, buf, len);
phdr->pd_lower += len;
MarkBufferDirty(buffer);
if (RelationNeedsWAL(rel))
{
XLogBeginInsert();
/*
* Since columnar will mostly write whole pages we force the transmission of the
* whole image in the buffer
*/
XLogRegisterBuffer(0, buffer, REGBUF_FORCE_IMAGE);
XLogRecPtr recptr = XLogInsert(RM_GENERIC_ID, 0);
PageSetLSN(page, recptr);
}
END_CRIT_SECTION();
UnlockReleaseBuffer(buffer);
}
/*
* AlignReservation - given an unused logical byte offset, align it so that it
* falls at the start of a page.
*
* XXX: Reconsider whether we want/need to do this at all.
*/
static uint64
AlignReservation(uint64 prevReservation)
{
PhysicalAddr prevAddr = LogicalToPhysical(prevReservation);
uint64 alignedReservation = prevReservation;
if (prevAddr.offset != SizeOfPageHeaderData)
{
/* not aligned; align on beginning of next page */
PhysicalAddr initial = { 0 };
initial.blockno = prevAddr.blockno + 1;
initial.offset = SizeOfPageHeaderData;
alignedReservation = PhysicalToLogical(initial);
}
Assert(alignedReservation >= prevReservation);
return alignedReservation;
}
/*
* ColumnarMetapageIsCurrent - is the metapage at the latest version?
*/
static bool
ColumnarMetapageIsCurrent(ColumnarMetapage *metapage)
{
return (metapage->versionMajor == COLUMNAR_VERSION_MAJOR &&
metapage->versionMinor == COLUMNAR_VERSION_MINOR);
}
/*
* ColumnarMetapageIsOlder - is the metapage older than the current version?
*/
static bool
ColumnarMetapageIsOlder(ColumnarMetapage *metapage)
{
return (metapage->versionMajor < COLUMNAR_VERSION_MAJOR ||
(metapage->versionMajor == COLUMNAR_VERSION_MAJOR &&
(int) metapage->versionMinor < (int) COLUMNAR_VERSION_MINOR));
}
/*
* ColumnarMetapageIsNewer - is the metapage newer than the current version?
*/
static bool
ColumnarMetapageIsNewer(ColumnarMetapage *metapage)
{
return (metapage->versionMajor > COLUMNAR_VERSION_MAJOR ||
(metapage->versionMajor == COLUMNAR_VERSION_MAJOR &&
metapage->versionMinor > COLUMNAR_VERSION_MINOR));
}
/*
* ColumnarMetapageCheckVersion - throw an error if accessing old
* version of metapage.
*/
static void
ColumnarMetapageCheckVersion(Relation rel, ColumnarMetapage *metapage)
{
if (!ColumnarMetapageIsCurrent(metapage))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg(
"attempted to access relation \"%s\", which uses an older columnar format",
RelationGetRelationName(rel)),
errdetail(
"Columnar format version %d.%d is required, \"%s\" has version %d.%d.",
COLUMNAR_VERSION_MAJOR, COLUMNAR_VERSION_MINOR,
RelationGetRelationName(rel),
metapage->versionMajor, metapage->versionMinor),
errhint(OLD_METAPAGE_VERSION_HINT)));
}
}

View File

@ -51,10 +51,12 @@
#include "columnar/columnar.h"
#include "columnar/columnar_customscan.h"
#include "columnar/columnar_storage.h"
#include "columnar/columnar_tableam.h"
#include "columnar/columnar_version_compat.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
/*
@ -81,12 +83,6 @@ typedef struct ColumnarScanDescData
MemoryContext scanContext;
Bitmapset *attr_needed;
List *scanQual;
/*
* ANALYZE requires an item pointer for sorting. We keep track of row
* number so we can construct an item pointer based on that.
*/
uint64 rowNumber;
} ColumnarScanDescData;
typedef struct ColumnarScanDescData *ColumnarScanDesc;
@ -115,6 +111,21 @@ static void TruncateColumnar(Relation rel, int elevel);
static HeapTuple ColumnarSlotCopyHeapTuple(TupleTableSlot *slot);
static void ColumnarCheckLogicalReplication(Relation rel);
static Datum * detoast_values(TupleDesc tupleDesc, Datum *orig_values, bool *isnull);
static ItemPointerData row_number_to_tid(uint64 rowNumber);
static uint64 tid_to_row_number(ItemPointerData tid);
static void ErrorIfInvalidRowNumber(uint64 rowNumber);
static void ColumnarReportTotalVirtualBlocks(Relation relation, Snapshot snapshot,
int progressArrIndex);
static BlockNumber ColumnarGetNumberOfVirtualBlocks(Relation relation, Snapshot snapshot);
static ItemPointerData ColumnarGetHighestItemPointer(Relation relation,
Snapshot snapshot);
static double ColumnarReadRowsIntoIndex(TableScanDesc scan,
Relation indexRelation,
IndexInfo *indexInfo,
bool progress,
IndexBuildCallback indexCallback,
void *indexCallbackState,
EState *estate, ExprState *predicate);
/* Custom tuple slot ops used for columnar. Initialized in columnar_tableam_init(). */
static TupleTableSlotOps TTSOpsColumnar;
@ -264,8 +275,9 @@ columnar_getnextslot(TableScanDesc sscan, ScanDirection direction, TupleTableSlo
ExecClearTuple(slot);
uint64 rowNumber;
bool nextRowFound = ColumnarReadNextRow(scan->cs_readState, slot->tts_values,
slot->tts_isnull);
slot->tts_isnull, &rowNumber);
if (!nextRowFound)
{
@ -274,65 +286,130 @@ columnar_getnextslot(TableScanDesc sscan, ScanDirection direction, TupleTableSlo
ExecStoreVirtualTuple(slot);
/*
* Set slot's item pointer block & offset to non-zero. These are
* used just for sorting in acquire_sample_rows(), so rowNumber
* is good enough. See ColumnarSlotCopyHeapTuple for more info.
*
* offset is 16-bits, so use the first 15 bits for offset and
* rest as block number.
*/
ItemPointerSetBlockNumber(&(slot->tts_tid), scan->rowNumber / (32 * 1024) + 1);
ItemPointerSetOffsetNumber(&(slot->tts_tid), scan->rowNumber % (32 * 1024) + 1);
scan->rowNumber++;
slot->tts_tid = row_number_to_tid(rowNumber);
return true;
}
/*
* row_number_to_tid maps given rowNumber to ItemPointerData.
*/
static ItemPointerData
row_number_to_tid(uint64 rowNumber)
{
ErrorIfInvalidRowNumber(rowNumber);
ItemPointerData tid = { 0 };
ItemPointerSetBlockNumber(&tid, rowNumber / VALID_ITEMPOINTER_OFFSETS);
ItemPointerSetOffsetNumber(&tid, rowNumber % VALID_ITEMPOINTER_OFFSETS +
FirstOffsetNumber);
return tid;
}
/*
* tid_to_row_number maps given ItemPointerData to rowNumber.
*/
static uint64
tid_to_row_number(ItemPointerData tid)
{
uint64 rowNumber = ItemPointerGetBlockNumber(&tid) * VALID_ITEMPOINTER_OFFSETS +
ItemPointerGetOffsetNumber(&tid) - FirstOffsetNumber;
ErrorIfInvalidRowNumber(rowNumber);
return rowNumber;
}
/*
* ErrorIfInvalidRowNumber errors out if given rowNumber is invalid.
*/
static void
ErrorIfInvalidRowNumber(uint64 rowNumber)
{
if (rowNumber == COLUMNAR_INVALID_ROW_NUMBER)
{
/* not expected but be on the safe side */
ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
errmsg("unexpected row number for columnar table")));
}
else if (rowNumber > COLUMNAR_MAX_ROW_NUMBER)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("columnar tables can't have row numbers "
"greater than " UINT64_FORMAT,
(uint64) COLUMNAR_MAX_ROW_NUMBER),
errhint("Consider using VACUUM FULL for your table")));
}
}
static Size
columnar_parallelscan_estimate(Relation rel)
{
elog(ERROR, "columnar_parallelscan_estimate not implemented");
return sizeof(ParallelBlockTableScanDescData);
}
static Size
columnar_parallelscan_initialize(Relation rel, ParallelTableScanDesc pscan)
{
elog(ERROR, "columnar_parallelscan_initialize not implemented");
ParallelBlockTableScanDesc bpscan = (ParallelBlockTableScanDesc) pscan;
bpscan->base.phs_relid = RelationGetRelid(rel);
bpscan->phs_nblocks = RelationGetNumberOfBlocks(rel);
bpscan->base.phs_syncscan = synchronize_seqscans &&
!RelationUsesLocalBuffers(rel) &&
bpscan->phs_nblocks > NBuffers / 4;
SpinLockInit(&bpscan->phs_mutex);
bpscan->phs_startblock = InvalidBlockNumber;
pg_atomic_init_u64(&bpscan->phs_nallocated, 0);
return sizeof(ParallelBlockTableScanDescData);
}
static void
columnar_parallelscan_reinitialize(Relation rel, ParallelTableScanDesc pscan)
{
elog(ERROR, "columnar_parallelscan_reinitialize not implemented");
ParallelBlockTableScanDesc bpscan = (ParallelBlockTableScanDesc) pscan;
pg_atomic_write_u64(&bpscan->phs_nallocated, 0);
}
static IndexFetchTableData *
columnar_index_fetch_begin(Relation rel)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
Oid relfilenode = rel->rd_node.relNode;
if (PendingWritesInUpperTransactions(relfilenode, GetCurrentSubTransactionId()))
{
/* XXX: maybe we can just flush the data and continue */
elog(ERROR, "cannot read from index when there is unflushed data in "
"upper transactions");
}
FlushWriteStateForRelfilenode(relfilenode, GetCurrentSubTransactionId());
IndexFetchTableData *scan = palloc0(sizeof(IndexFetchTableData));
scan->rel = rel;
return scan;
}
static void
columnar_index_fetch_reset(IndexFetchTableData *scan)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
/* no-op */
}
static void
columnar_index_fetch_end(IndexFetchTableData *scan)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
columnar_index_fetch_reset(scan);
pfree(scan);
}
@ -343,8 +420,37 @@ columnar_index_fetch_tuple(struct IndexFetchTableData *scan,
TupleTableSlot *slot,
bool *call_again, bool *all_dead)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
/* no HOT chains are possible in columnar, directly set it to false */
*call_again = false;
/*
* No dead tuples are possible in columnar, set it to false if it's
* passed to be non-NULL.
*/
if (all_dead)
{
*all_dead = false;
}
ExecClearTuple(slot);
/* we need all columns */
int natts = scan->rel->rd_att->natts;
Bitmapset *attr_needed = bms_add_range(NULL, 0, natts - 1);
TupleDesc relationTupleDesc = RelationGetDescr(scan->rel);
List *relationColumnList = NeededColumnsList(relationTupleDesc, attr_needed);
uint64 rowNumber = tid_to_row_number(*tid);
if (!ColumnarReadRowByRowNumber(scan->rel, rowNumber, relationColumnList,
slot->tts_values, slot->tts_isnull, snapshot))
{
return false;
}
slot->tts_tableOid = RelationGetRelid(scan->rel);
slot->tts_tid = *tid;
ExecStoreVirtualTuple(slot);
return true;
}
@ -411,7 +517,8 @@ columnar_tuple_insert(Relation relation, TupleTableSlot *slot, CommandId cid,
Datum *values = detoast_values(slot->tts_tupleDescriptor,
slot->tts_values, slot->tts_isnull);
ColumnarWriteRow(writeState, values, slot->tts_isnull);
uint64 writtenRowNumber = ColumnarWriteRow(writeState, values, slot->tts_isnull);
slot->tts_tid = row_number_to_tid(writtenRowNumber);
MemoryContextSwitchTo(oldContext);
MemoryContextReset(ColumnarWritePerTupleContext(writeState));
@ -457,7 +564,10 @@ columnar_multi_insert(Relation relation, TupleTableSlot **slots, int ntuples,
Datum *values = detoast_values(tupleSlot->tts_tupleDescriptor,
tupleSlot->tts_values, tupleSlot->tts_isnull);
ColumnarWriteRow(writeState, values, tupleSlot->tts_isnull);
uint64 writtenRowNumber = ColumnarWriteRow(writeState, values,
tupleSlot->tts_isnull);
tupleSlot->tts_tid = row_number_to_tid(writtenRowNumber);
MemoryContextReset(ColumnarWritePerTupleContext(writeState));
}
@ -516,17 +626,24 @@ columnar_relation_set_new_filenode(Relation rel,
errmsg("unlogged columnar tables are not supported")));
}
Oid oldRelfilenode = rel->rd_node.relNode;
/*
* If existing and new relfilenode are different, that means the existing
* storage was dropped and we also need to clean up the metadata and
* state. If they are equal, this is a new relation object and we don't
* need to clean anything.
*/
if (rel->rd_node.relNode != newrnode->relNode)
{
MarkRelfilenodeDropped(rel->rd_node.relNode, GetCurrentSubTransactionId());
MarkRelfilenodeDropped(oldRelfilenode, GetCurrentSubTransactionId());
/* delete old relfilenode metadata */
DeleteMetadataRows(rel->rd_node);
DeleteMetadataRows(rel->rd_node);
}
*freezeXid = RecentXmin;
*minmulti = GetOldestMultiXactId();
SMgrRelation srel = RelationCreateStorage(*newrnode, persistence);
ColumnarStorageInit(srel, ColumnarMetadataNewStorageId());
InitColumnarOptions(rel->rd_id);
smgrclose(srel);
@ -554,7 +671,9 @@ columnar_relation_nontransactional_truncate(Relation rel)
*/
RelationTruncate(rel, 0);
/* we will lazily initialize new metadata in first stripe reservation */
uint64 storageId = ColumnarMetadataNewStorageId();
RelationOpenSmgr(rel);
ColumnarStorageInit(rel->rd_smgr, storageId);
}
@ -588,7 +707,8 @@ columnar_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
if (OldIndex != NULL || use_sort)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
errmsg("clustering columnar tables using indexes is "
"not supported")));
}
/*
@ -619,7 +739,8 @@ columnar_relation_copy_for_cluster(Relation OldHeap, Relation NewHeap,
*num_tuples = 0;
while (ColumnarReadNextRow(readState, values, nulls))
/* we don't need to know rowNumber here */
while (ColumnarReadNextRow(readState, values, nulls, NULL))
{
ColumnarWriteRow(writeState, values, nulls);
(*num_tuples)++;
@ -667,6 +788,14 @@ static void
columnar_vacuum_rel(Relation rel, VacuumParams *params,
BufferAccessStrategy bstrategy)
{
/*
* If metapage version of relation is older, then we hint users to VACUUM
* the relation in ColumnarMetapageCheckVersion. So if needed, upgrade
* the metapage before doing anything.
*/
bool isUpgrade = true;
ColumnarStorageUpdateIfNeeded(rel, isUpgrade);
int elevel = (params->options & VACOPT_VERBOSE) ? INFO : DEBUG2;
/* this should have been resolved by vacuum.c until now */
@ -840,34 +969,25 @@ TruncateColumnar(Relation rel, int elevel)
return;
}
RelationOpenSmgr(rel);
BlockNumber old_rel_pages = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
RelationCloseSmgr(rel);
/*
* Due to the AccessExclusive lock there's no danger that
* new stripes be added beyond highestPhysicalAddress while
* we're truncating.
*/
SmgrAddr highestPhysicalAddress =
logical_to_smgr(GetHighestUsedAddress(rel->rd_node));
uint64 newDataReservation = Max(GetHighestUsedAddress(rel->rd_node) + 1,
ColumnarFirstLogicalOffset);
/*
* Unlock and return if truncation won't reduce data file's size.
*/
BlockNumber new_rel_pages = Min(old_rel_pages,
highestPhysicalAddress.blockno + 1);
if (new_rel_pages == old_rel_pages)
RelationOpenSmgr(rel);
BlockNumber old_rel_pages = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
if (!ColumnarStorageTruncate(rel, newDataReservation))
{
UnlockRelation(rel, AccessExclusiveLock);
return;
}
/*
* Truncate the storage. Note that RelationTruncate() takes care of
* Write Ahead Logging.
*/
RelationTruncate(rel, new_rel_pages);
RelationOpenSmgr(rel);
BlockNumber new_rel_pages = smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
/*
* We can release the exclusive lock as soon as we have truncated.
@ -964,7 +1084,7 @@ columnar_scan_analyze_next_tuple(TableScanDesc scan, TransactionId OldestXmin,
static double
columnar_index_build_range_scan(Relation heapRelation,
columnar_index_build_range_scan(Relation columnarRelation,
Relation indexRelation,
IndexInfo *indexInfo,
bool allow_sync,
@ -976,8 +1096,278 @@ columnar_index_build_range_scan(Relation heapRelation,
void *callback_state,
TableScanDesc scan)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
if (start_blockno != 0 || numblocks != InvalidBlockNumber)
{
/*
* Columnar utility hook already errors out for BRIN indexes on columnar
* tables, but be on the safe side.
*/
ereport(ERROR, (errmsg("BRIN indexes on columnar tables are not supported")));
}
if (indexInfo->ii_Concurrent)
{
/* we already don't allow CONCURRENTLY syntax but be on the safe side */
ereport(ERROR, (errmsg("concurrent index builds are not supported "
"for columnar tables")));
}
if (scan)
{
/*
* Scan is initialized iff postgres decided to build the index using
* parallel workers. In this case, we simply return for parallel
* workers since we don't support parallel scan on columnar tables.
*/
if (IsBackgroundWorker)
{
ereport(DEBUG4, (errmsg("ignoring parallel worker when building "
"index since parallel scan on columnar "
"tables is not supported")));
return 0;
}
ereport(NOTICE, (errmsg("falling back to serial index build since "
"parallel scan on columnar tables is not "
"supported")));
}
/*
* In a normal index build, we use SnapshotAny to retrieve all tuples. In
* a concurrent build or during bootstrap, we take a regular MVCC snapshot
* and index whatever's live according to that.
*/
TransactionId OldestXmin = InvalidTransactionId;
/*
* We already don't allow concurrent index builds so ii_Concurrent
* will always be false, but let's keep the code close to heapAM.
*/
if (!IsBootstrapProcessingMode() && !indexInfo->ii_Concurrent)
{
/* ignore lazy VACUUM's */
OldestXmin = GetOldestXmin(columnarRelation, PROCARRAY_FLAGS_VACUUM);
}
Snapshot snapshot = { 0 };
bool snapshotRegisteredByUs = false;
if (!scan)
{
/*
* For serial index build, we begin our own scan. We may also need to
* register a snapshot whose lifetime is under our direct control.
*/
if (!TransactionIdIsValid(OldestXmin))
{
snapshot = RegisterSnapshot(GetTransactionSnapshot());
snapshotRegisteredByUs = true;
}
else
{
snapshot = SnapshotAny;
}
int nkeys = 0;
ScanKeyData *scanKey = NULL;
bool allowAccessStrategy = true;
scan = table_beginscan_strat(columnarRelation, snapshot, nkeys, scanKey,
allowAccessStrategy, allow_sync);
}
else
{
/*
* For parallel index build, we don't register/unregister own snapshot
* since snapshot is taken from parallel scan. Note that even if we
* don't support parallel index builds, we still continue building the
* index via the main backend and we should still rely on the snapshot
* provided by parallel scan.
*/
snapshot = scan->rs_snapshot;
}
if (progress)
{
ColumnarReportTotalVirtualBlocks(columnarRelation, snapshot,
PROGRESS_SCAN_BLOCKS_TOTAL);
}
/*
* Set up execution state for predicate, if any.
* Note that this is only useful for partial indexes.
*/
EState *estate = CreateExecutorState();
ExprContext *econtext = GetPerTupleExprContext(estate);
econtext->ecxt_scantuple = table_slot_create(columnarRelation, NULL);
ExprState *predicate = ExecPrepareQual(indexInfo->ii_Predicate, estate);
double reltuples = ColumnarReadRowsIntoIndex(scan, indexRelation, indexInfo,
progress, callback, callback_state,
estate, predicate);
table_endscan(scan);
if (progress)
{
/* report the last "virtual" block as "done" */
ColumnarReportTotalVirtualBlocks(columnarRelation, snapshot,
PROGRESS_SCAN_BLOCKS_DONE);
}
if (snapshotRegisteredByUs)
{
UnregisterSnapshot(snapshot);
}
ExecDropSingleTupleTableSlot(econtext->ecxt_scantuple);
FreeExecutorState(estate);
indexInfo->ii_ExpressionsState = NIL;
indexInfo->ii_PredicateState = NULL;
return reltuples;
}
/*
* ColumnarReportTotalVirtualBlocks reports progress for index build based on
* number of "virtual" blocks that given relation has.
* "progressArrIndex" argument determines which entry in st_progress_param
* array should be updated. In this case, we only expect PROGRESS_SCAN_BLOCKS_TOTAL
* or PROGRESS_SCAN_BLOCKS_DONE to specify whether we want to report calculated
* number of blocks as "done" or as "total" number of "virtual" blocks to scan.
*/
static void
ColumnarReportTotalVirtualBlocks(Relation relation, Snapshot snapshot,
int progressArrIndex)
{
/*
* Indeed, columnar tables might have gaps between row numbers, e.g
* due to aborted transactions etc. Also, ItemPointer BlockNumber's
* for columnar tables don't actually correspond to actual disk blocks
* as in heapAM. For this reason, we call them as "virtual" blocks. At
* the moment, we believe it is better to report our progress based on
* this "virtual" block concept instead of doing nothing.
*/
Assert(progressArrIndex == PROGRESS_SCAN_BLOCKS_TOTAL ||
progressArrIndex == PROGRESS_SCAN_BLOCKS_DONE);
BlockNumber nvirtualBlocks =
ColumnarGetNumberOfVirtualBlocks(relation, snapshot);
pgstat_progress_update_param(progressArrIndex, nvirtualBlocks);
}
/*
* ColumnarGetNumberOfVirtualBlocks returns total number of "virtual" blocks
* that given columnar table has based on based on ItemPointer BlockNumber's.
*/
static BlockNumber
ColumnarGetNumberOfVirtualBlocks(Relation relation, Snapshot snapshot)
{
ItemPointerData highestItemPointer =
ColumnarGetHighestItemPointer(relation, snapshot);
if (!ItemPointerIsValid(&highestItemPointer))
{
/* table is empty according to our snapshot */
return 0;
}
/*
* Since BlockNumber is 0-based, increment it by 1 to find the total
* number of "virtual" blocks.
*/
return ItemPointerGetBlockNumber(&highestItemPointer) + 1;
}
/*
* ColumnarGetHighestItemPointer returns ItemPointerData for the tuple with
* highest tid for given relation.
* If given relation is empty, then returns invalid item pointer.
*/
static ItemPointerData
ColumnarGetHighestItemPointer(Relation relation, Snapshot snapshot)
{
StripeMetadata *stripeWithHighestRowNumber =
FindStripeWithHighestRowNumber(relation, snapshot);
if (stripeWithHighestRowNumber == NULL)
{
/* table is empty according to our snapshot */
ItemPointerData invalidItemPtr;
ItemPointerSetInvalid(&invalidItemPtr);
return invalidItemPtr;
}
uint64 highestRowNumber = stripeWithHighestRowNumber->firstRowNumber +
stripeWithHighestRowNumber->rowCount - 1;
return row_number_to_tid(highestRowNumber);
}
/*
* ColumnarReadRowsIntoIndex builds indexRelation tuples by reading the
* actual relation based on given "scan" and returns number of tuples
* scanned to build the indexRelation.
*/
static double
ColumnarReadRowsIntoIndex(TableScanDesc scan, Relation indexRelation,
IndexInfo *indexInfo, bool progress,
IndexBuildCallback indexCallback,
void *indexCallbackState, EState *estate,
ExprState *predicate)
{
double reltuples = 0;
BlockNumber lastReportedBlockNumber = InvalidBlockNumber;
ExprContext *econtext = GetPerTupleExprContext(estate);
TupleTableSlot *slot = econtext->ecxt_scantuple;
while (columnar_getnextslot(scan, ForwardScanDirection, slot))
{
CHECK_FOR_INTERRUPTS();
BlockNumber currentBlockNumber = ItemPointerGetBlockNumber(&slot->tts_tid);
if (progress && lastReportedBlockNumber != currentBlockNumber)
{
/*
* columnar_getnextslot guarantees that returned tuple will
* always have a greater ItemPointer than the ones we fetched
* before, so we directly use BlockNumber to report our progress.
*/
Assert(lastReportedBlockNumber == InvalidBlockNumber ||
currentBlockNumber >= lastReportedBlockNumber);
pgstat_progress_update_param(PROGRESS_SCAN_BLOCKS_DONE,
currentBlockNumber);
lastReportedBlockNumber = currentBlockNumber;
}
MemoryContextReset(econtext->ecxt_per_tuple_memory);
if (predicate != NULL && !ExecQual(predicate, econtext))
{
/* for partial indexes, discard tuples that don't satisfy the predicate */
continue;
}
Datum indexValues[INDEX_MAX_KEYS];
bool indexNulls[INDEX_MAX_KEYS];
FormIndexDatum(indexInfo, slot, estate, indexValues, indexNulls);
ItemPointerData itemPointerData = slot->tts_tid;
/* currently, columnar tables can't have dead tuples */
bool tupleIsAlive = true;
#if PG_VERSION_NUM >= PG_VERSION_13
indexCallback(indexRelation, &itemPointerData, indexValues, indexNulls,
tupleIsAlive, indexCallbackState);
#else
HeapTuple scanTuple = ExecCopySlotHeapTuple(slot);
scanTuple->t_self = itemPointerData;
indexCallback(indexRelation, scanTuple, indexValues, indexNulls,
tupleIsAlive, indexCallbackState);
#endif
reltuples++;
}
return reltuples;
}
@ -988,8 +1378,15 @@ columnar_index_validate_scan(Relation heapRelation,
Snapshot snapshot,
ValidateIndexState *state)
{
/*
* This is only called for concurrent index builds,
* see table_index_validate_scan.
* Note that we already error out for concurrent index
* builds in utility hook but be on the safe side.
*/
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
errmsg("concurrent index builds are not supported for "
"columnar tables")));
}
@ -1179,13 +1576,7 @@ ColumnarSlotCopyHeapTuple(TupleTableSlot *slot)
slot->tts_values,
slot->tts_isnull);
/*
* We need to set item pointer, since implementation of ANALYZE
* requires it. See the qsort in acquire_sample_rows() and
* also compare_rows in backend/commands/analyze.c.
*
* slot->tts_tid is filled in columnar_getnextslot.
*/
/* slot->tts_tid is filled in columnar_getnextslot */
tuple->t_self = slot->tts_tid;
return tuple;
@ -1315,24 +1706,34 @@ ColumnarProcessUtility(PlannedStmt *pstmt,
{
IndexStmt *indexStmt = (IndexStmt *) parsetree;
/*
* We should reject CREATE INDEX CONCURRENTLY before DefineIndex() is
* called. Erroring in callbacks called from DefineIndex() will create
* the index and mark it as INVALID, which will cause segfault during
* inserts.
*/
if (indexStmt->concurrent)
Relation rel = relation_openrv(indexStmt->relation,
GetCreateIndexRelationLockMode(indexStmt));
if (rel->rd_tableam == GetColumnarTableAmRoutine())
{
Relation rel = relation_openrv(indexStmt->relation,
ShareUpdateExclusiveLock);
if (rel->rd_tableam == GetColumnarTableAmRoutine())
/*
* We should reject CREATE INDEX CONCURRENTLY before DefineIndex() is
* called. Erroring in callbacks called from DefineIndex() will create
* the index and mark it as INVALID, which will cause segfault during
* inserts.
*/
if (indexStmt->concurrent)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("indexes not supported for columnar tables")));
errmsg("concurrent index commands are not "
"supported for columnar tables")));
}
RelationClose(rel);
/* for now, we don't support index access methods other than btree & hash */
if (strncmp(indexStmt->accessMethod, "btree", NAMEDATALEN) != 0 &&
strncmp(indexStmt->accessMethod, "hash", NAMEDATALEN) != 0)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("only btree and hash indexes are supported on "
"columnar tables ")));
}
}
RelationClose(rel);
}
PrevProcessUtilityHook(pstmt, queryString, context,
@ -1374,6 +1775,17 @@ static const TableAmRoutine columnar_am_methods = {
.scan_rescan = columnar_rescan,
.scan_getnextslot = columnar_getnextslot,
/*
* Postgres calls following three callbacks during index builds, if it
* decides to use parallel workers when building the index. On the other
* hand, we don't support parallel scans on columnar tables but we also
* want to fallback to serial index build. For this reason, we both skip
* parallel workers in columnar_index_build_range_scan and also provide
* basic implementations for those callbacks based on their corresponding
* implementations in heapAM.
* Note that for regular query plans, we already ignore parallel paths via
* ColumnarSetRelPathlistHook.
*/
.parallelscan_estimate = columnar_parallelscan_estimate,
.parallelscan_initialize = columnar_parallelscan_initialize,
.parallelscan_reinitialize = columnar_parallelscan_reinitialize,
@ -1840,3 +2252,75 @@ alter_columnar_table_reset(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
/*
* upgrade_columnar_storage - upgrade columnar storage to the current
* version.
*
* DDL:
* CREATE OR REPLACE FUNCTION upgrade_columnar_storage(rel regclass)
* RETURNS VOID
* STRICT
* LANGUAGE c AS 'MODULE_PATHNAME', 'upgrade_columnar_storage';
*/
PG_FUNCTION_INFO_V1(upgrade_columnar_storage);
Datum
upgrade_columnar_storage(PG_FUNCTION_ARGS)
{
Oid relid = PG_GETARG_OID(0);
/*
* ACCESS EXCLUSIVE LOCK is not required by the low-level routines, so we
* can take only an ACCESS SHARE LOCK. But all access to non-current
* columnar tables will fail anyway, so it's better to take ACCESS
* EXLUSIVE LOCK now.
*/
Relation rel = table_open(relid, AccessExclusiveLock);
if (!IsColumnarTableAmTable(relid))
{
ereport(ERROR, (errmsg("table %s is not a columnar table",
quote_identifier(RelationGetRelationName(rel)))));
}
ColumnarStorageUpdateIfNeeded(rel, true);
table_close(rel, AccessExclusiveLock);
PG_RETURN_VOID();
}
/*
* downgrade_columnar_storage - downgrade columnar storage to the
* current version.
*
* DDL:
* CREATE OR REPLACE FUNCTION downgrade_columnar_storage(rel regclass)
* RETURNS VOID
* STRICT
* LANGUAGE c AS 'MODULE_PATHNAME', 'downgrade_columnar_storage';
*/
PG_FUNCTION_INFO_V1(downgrade_columnar_storage);
Datum
downgrade_columnar_storage(PG_FUNCTION_ARGS)
{
Oid relid = PG_GETARG_OID(0);
/*
* ACCESS EXCLUSIVE LOCK is not required by the low-level routines, so we
* can take only an ACCESS SHARE LOCK. But all access to non-current
* columnar tables will fail anyway, so it's better to take ACCESS
* EXLUSIVE LOCK now.
*/
Relation rel = table_open(relid, AccessExclusiveLock);
if (!IsColumnarTableAmTable(relid))
{
ereport(ERROR, (errmsg("table %s is not a columnar table",
quote_identifier(RelationGetRelationName(rel)))));
}
ColumnarStorageUpdateIfNeeded(rel, false);
table_close(rel, AccessExclusiveLock);
PG_RETURN_VOID();
}

View File

@ -30,6 +30,7 @@
#include "utils/relfilenodemap.h"
#include "columnar/columnar.h"
#include "columnar/columnar_storage.h"
#include "columnar/columnar_version_compat.h"
struct ColumnarWriteState
@ -42,6 +43,7 @@ struct ColumnarWriteState
MemoryContext perTupleContext;
StripeBuffers *stripeBuffers;
StripeSkipList *stripeSkipList;
uint64 stripeFirstRowNumber;
ColumnarOptions options;
ChunkData *chunkData;
@ -128,6 +130,7 @@ ColumnarBeginWrite(RelFileNode relfilenode,
writeState->comparisonFunctionArray = comparisonFunctionArray;
writeState->stripeBuffers = NULL;
writeState->stripeSkipList = NULL;
writeState->stripeFirstRowNumber = COLUMNAR_INVALID_ROW_NUMBER;
writeState->stripeWriteContext = stripeWriteContext;
writeState->chunkData = chunkData;
writeState->compressionBuffer = NULL;
@ -146,8 +149,10 @@ ColumnarBeginWrite(RelFileNode relfilenode,
* corresponding skip nodes. Then, whole chunk data is compressed at every
* rowChunkCount insertion. Then, if row count exceeds stripeMaxRowCount, we flush
* the stripe, and add its metadata to the table footer.
*
* Returns the "row number" assigned to written row.
*/
void
uint64
ColumnarWriteRow(ColumnarWriteState *writeState, Datum *columnValues, bool *columnNulls)
{
uint32 columnIndex = 0;
@ -169,6 +174,14 @@ ColumnarWriteRow(ColumnarWriteState *writeState, Datum *columnValues, bool *colu
writeState->stripeSkipList = stripeSkipList;
writeState->compressionBuffer = makeStringInfo();
Oid relationId = RelidByRelfilenode(writeState->relfilenode.spcNode,
writeState->relfilenode.relNode);
Relation relation = relation_open(relationId, NoLock);
writeState->stripeFirstRowNumber =
ColumnarStorageReserveRowNumber(relation,
options->stripeRowCount);
relation_close(relation, NoLock);
/*
* serializedValueBuffer lives in stripe write memory context so it needs to be
* initialized when the stripe is created.
@ -225,6 +238,7 @@ ColumnarWriteRow(ColumnarWriteState *writeState, Datum *columnValues, bool *colu
SerializeChunkData(writeState, chunkIndex, chunkRowCount);
}
uint64 writtenRowNumber = writeState->stripeFirstRowNumber + stripeBuffers->rowCount;
stripeBuffers->rowCount++;
if (stripeBuffers->rowCount >= options->stripeRowCount)
{
@ -232,6 +246,8 @@ ColumnarWriteRow(ColumnarWriteState *writeState, Datum *columnValues, bool *colu
}
MemoryContextSwitchTo(oldContext);
return writtenRowNumber;
}
@ -351,80 +367,6 @@ CreateEmptyStripeSkipList(uint32 stripeMaxRowCount, uint32 chunkRowCount,
}
void
WriteToSmgr(Relation rel, uint64 logicalOffset, char *data, uint32 dataLength)
{
uint64 remaining = dataLength;
Buffer buffer;
while (remaining > 0)
{
SmgrAddr addr = logical_to_smgr(logicalOffset);
RelationOpenSmgr(rel);
BlockNumber nblocks PG_USED_FOR_ASSERTS_ONLY =
smgrnblocks(rel->rd_smgr, MAIN_FORKNUM);
Assert(addr.blockno < nblocks);
RelationCloseSmgr(rel);
buffer = ReadBuffer(rel, addr.blockno);
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
Page page = BufferGetPage(buffer);
PageHeader phdr = (PageHeader) page;
if (PageIsNew(page))
{
PageInit(page, BLCKSZ, 0);
}
/*
* After a transaction has been rolled-back, we might be
* over-writing the rolledback write, so phdr->pd_lower can be
* different from addr.offset.
*
* We reset pd_lower to reset the rolledback write.
*/
if (phdr->pd_lower > addr.offset)
{
ereport(DEBUG1, (errmsg("over-writing page %u", addr.blockno),
errdetail("This can happen after a roll-back.")));
phdr->pd_lower = addr.offset;
}
Assert(phdr->pd_lower == addr.offset);
START_CRIT_SECTION();
uint64 to_write = Min(phdr->pd_upper - phdr->pd_lower, remaining);
memcpy_s(page + phdr->pd_lower, phdr->pd_upper - phdr->pd_lower, data, to_write);
phdr->pd_lower += to_write;
MarkBufferDirty(buffer);
if (RelationNeedsWAL(rel))
{
XLogBeginInsert();
/*
* Since columnar will mostly write whole pages we force the transmission of the
* whole image in the buffer
*/
XLogRegisterBuffer(0, buffer, REGBUF_FORCE_IMAGE);
XLogRecPtr recptr = XLogInsert(RM_GENERIC_ID, 0);
PageSetLSN(page, recptr);
}
END_CRIT_SECTION();
UnlockReleaseBuffer(buffer);
data += to_write;
remaining -= to_write;
logicalOffset += to_write;
}
}
/*
* FlushStripe flushes current stripe data into the file. The function first ensures
* the last data chunk for each column is properly serialized and compressed. Then,
@ -502,7 +444,7 @@ FlushStripe(ColumnarWriteState *writeState)
stripeMetadata = ReserveStripe(relation, stripeSize,
stripeRowCount, columnCount, chunkCount,
chunkRowCount);
chunkRowCount, writeState->stripeFirstRowNumber);
uint64 currentFileOffset = stripeMetadata.fileOffset;
@ -527,8 +469,8 @@ FlushStripe(ColumnarWriteState *writeState)
columnBuffers->chunkBuffersArray[chunkIndex];
StringInfo existsBuffer = chunkBuffers->existsBuffer;
WriteToSmgr(relation, currentFileOffset,
existsBuffer->data, existsBuffer->len);
ColumnarStorageWrite(relation, currentFileOffset,
existsBuffer->data, existsBuffer->len);
currentFileOffset += existsBuffer->len;
}
@ -538,8 +480,8 @@ FlushStripe(ColumnarWriteState *writeState)
columnBuffers->chunkBuffersArray[chunkIndex];
StringInfo valueBuffer = chunkBuffers->valueBuffer;
WriteToSmgr(relation, currentFileOffset,
valueBuffer->data, valueBuffer->len);
ColumnarStorageWrite(relation, currentFileOffset,
valueBuffer->data, valueBuffer->len);
currentFileOffset += valueBuffer->len;
}
}

View File

@ -0,0 +1,32 @@
/* columnar--10.1-1--10.2-1.sql */
-- For a proper mapping between tid & (stripe, row_num), add a new column to
-- columnar.stripe and define a BTREE index on this column.
-- Also include storage_id column for per-relation scans.
ALTER TABLE columnar.stripe ADD COLUMN first_row_number bigint;
CREATE INDEX stripe_first_row_number_idx ON columnar.stripe USING BTREE(storage_id, first_row_number);
-- Populate first_row_number column of columnar.stripe table.
--
-- For simplicity, we calculate MAX(row_count) value across all the stripes
-- of all the columanar tables and then use it to populate first_row_number
-- column. This would introduce some gaps however we are okay with that since
-- it's already the case with regular INSERT/COPY's.
DO $$
DECLARE
max_row_count bigint;
-- this should be equal to columnar_storage.h/COLUMNAR_FIRST_ROW_NUMBER
COLUMNAR_FIRST_ROW_NUMBER constant bigint := 1;
BEGIN
SELECT MAX(row_count) INTO max_row_count FROM columnar.stripe;
UPDATE columnar.stripe SET first_row_number = COLUMNAR_FIRST_ROW_NUMBER +
(stripe_num - 1) * max_row_count;
END;
$$;
#include "udfs/upgrade_columnar_storage/10.2-1.sql"
#include "udfs/downgrade_columnar_storage/10.2-1.sql"
-- upgrade storage for all columnar relations
SELECT citus_internal.upgrade_columnar_storage(c.oid) FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar';

View File

@ -0,0 +1,12 @@
/* columnar--10.2-1--10.1-1.sql */
-- downgrade storage for all columnar relations
SELECT citus_internal.downgrade_columnar_storage(c.oid) FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar';
DROP FUNCTION citus_internal.upgrade_columnar_storage(regclass);
DROP FUNCTION citus_internal.downgrade_columnar_storage(regclass);
-- drop "first_row_number" column and the index defined on it
DROP INDEX columnar.stripe_first_row_number_idx;
ALTER TABLE columnar.stripe DROP COLUMN first_row_number;

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.downgrade_columnar_storage(rel regclass)
RETURNS VOID
STRICT
LANGUAGE c AS 'MODULE_PATHNAME', $$downgrade_columnar_storage$$;
COMMENT ON FUNCTION citus_internal.downgrade_columnar_storage(regclass)
IS 'function to downgrade the columnar storage, if necessary';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.downgrade_columnar_storage(rel regclass)
RETURNS VOID
STRICT
LANGUAGE c AS 'MODULE_PATHNAME', $$downgrade_columnar_storage$$;
COMMENT ON FUNCTION citus_internal.downgrade_columnar_storage(regclass)
IS 'function to downgrade the columnar storage, if necessary';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.upgrade_columnar_storage(rel regclass)
RETURNS VOID
STRICT
LANGUAGE c AS 'MODULE_PATHNAME', $$upgrade_columnar_storage$$;
COMMENT ON FUNCTION citus_internal.upgrade_columnar_storage(regclass)
IS 'function to upgrade the columnar storage, if necessary';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.upgrade_columnar_storage(rel regclass)
RETURNS VOID
STRICT
LANGUAGE c AS 'MODULE_PATHNAME', $$upgrade_columnar_storage$$;
COMMENT ON FUNCTION citus_internal.upgrade_columnar_storage(regclass)
IS 'function to upgrade the columnar storage, if necessary';

View File

@ -62,7 +62,6 @@ static List * GenerateIndexParameters(IndexStmt *createIndexStatement);
static DDLJob * GenerateCreateIndexDDLJob(IndexStmt *createIndexStatement,
const char *createIndexCommand);
static Oid CreateIndexStmtGetRelationId(IndexStmt *createIndexStatement);
static LOCKMODE GetCreateIndexRelationLockMode(IndexStmt *createIndexStatement);
static List * CreateIndexTaskList(IndexStmt *indexStmt);
static List * CreateReindexTaskList(Oid relationId, ReindexStmt *reindexStmt);
static void RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid,
@ -503,7 +502,7 @@ CreateIndexStmtGetRelationId(IndexStmt *createIndexStatement)
* GetCreateIndexRelationLockMode returns required lock mode to open the
* relation that given CREATE INDEX command operates on.
*/
static LOCKMODE
LOCKMODE
GetCreateIndexRelationLockMode(IndexStmt *createIndexStatement)
{
if (createIndexStatement->concurrent)

View File

@ -1,4 +1,3 @@
-- citus--10.1-1--10.2-1
-- bump version to 10.2-1
#include "../../columnar/sql/columnar--10.1-1--10.2-1.sql"

View File

@ -1,2 +1,3 @@
-- citus--10.2-1--10.1-1
-- this is an empty downgrade path since citus--10.1-1--10.2-1.sql is empty for now
#include "../../../columnar/sql/downgrades/columnar--10.2-1--10.1-1.sql"

View File

@ -25,7 +25,6 @@
#include "distributed/lock_graph.h"
#include "distributed/metadata_cache.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/tuplestore.h"
#include "storage/proc.h"
#include "utils/builtins.h"
@ -472,18 +471,9 @@ IsProcessWaitingForSafeOperations(PGPROC *proc)
PROCLOCK *waitProcLock = proc->waitProcLock;
LOCK *waitLock = waitProcLock->tag.myLock;
/*
* Stripe reservation locks are temporary & don't hold until end of
* transaction, so we shouldn't include them in the lock graph.
*/
bool stripeReservationLock =
waitLock->tag.locktag_type == LOCKTAG_ADVISORY &&
waitLock->tag.locktag_field4 == ADV_LOCKTAG_CLASS_COLUMNAR_STRIPE_RESERVATION;
return waitLock->tag.locktag_type == LOCKTAG_RELATION_EXTEND ||
waitLock->tag.locktag_type == LOCKTAG_PAGE ||
waitLock->tag.locktag_type == LOCKTAG_SPECULATIVE_TOKEN ||
stripeReservationLock;
waitLock->tag.locktag_type == LOCKTAG_SPECULATIVE_TOKEN;
}

View File

@ -39,8 +39,8 @@
#define COMPRESSION_LEVEL_MAX 19
/* Columnar file signature */
#define COLUMNAR_VERSION_MAJOR 1
#define COLUMNAR_VERSION_MINOR 7
#define COLUMNAR_VERSION_MAJOR 2
#define COLUMNAR_VERSION_MINOR 0
/* miscellaneous defines */
#define COLUMNAR_TUPLE_COST_MULTIPLIER 10
@ -201,8 +201,8 @@ extern CompressionType ParseCompressionType(const char *compressionTypeString);
extern ColumnarWriteState * ColumnarBeginWrite(RelFileNode relfilenode,
ColumnarOptions options,
TupleDesc tupleDescriptor);
extern void ColumnarWriteRow(ColumnarWriteState *state, Datum *columnValues,
bool *columnNulls);
extern uint64 ColumnarWriteRow(ColumnarWriteState *state, Datum *columnValues,
bool *columnNulls);
extern void ColumnarFlushPendingWrites(ColumnarWriteState *state);
extern void ColumnarEndWrite(ColumnarWriteState *state);
extern bool ContainsPendingWrites(ColumnarWriteState *state);
@ -214,8 +214,11 @@ extern ColumnarReadState * ColumnarBeginRead(Relation relation,
List *projectedColumnList,
List *qualConditions);
extern bool ColumnarReadNextRow(ColumnarReadState *state, Datum *columnValues,
bool *columnNulls);
bool *columnNulls, uint64 *rowNumber);
extern void ColumnarRescan(ColumnarReadState *readState);
extern bool ColumnarReadRowByRowNumber(Relation relation, uint64 rowNumber,
List *neededColumnList, Datum *columnValues,
bool *columnNulls, Snapshot snapshot);
extern void ColumnarEndRead(ColumnarReadState *state);
extern int64 ColumnarReadChunkGroupsFiltered(ColumnarReadState *state);
@ -233,17 +236,16 @@ extern void InitColumnarOptions(Oid regclass);
extern void SetColumnarOptions(Oid regclass, ColumnarOptions *options);
extern bool DeleteColumnarTableOptions(Oid regclass, bool missingOk);
extern bool ReadColumnarOptions(Oid regclass, ColumnarOptions *options);
extern void WriteToSmgr(Relation relation, uint64 logicalOffset,
char *data, uint32 dataLength);
extern StringInfo ReadFromSmgr(Relation rel, uint64 offset, uint32 size);
extern bool IsColumnarTableAmTable(Oid relationId);
/* columnar_metadata_tables.c */
extern void DeleteMetadataRows(RelFileNode relfilenode);
extern uint64 ColumnarMetadataNewStorageId(void);
extern uint64 GetHighestUsedAddress(RelFileNode relfilenode);
extern StripeMetadata ReserveStripe(Relation rel, uint64 size,
uint64 rowCount, uint64 columnCount,
uint64 chunkCount, uint64 chunkGroupRowCount);
uint64 chunkCount, uint64 chunkGroupRowCount,
uint64 stripeFirstRowNumber);
extern void SaveStripeSkipList(RelFileNode relfilenode, uint64 stripe,
StripeSkipList *stripeSkipList,
TupleDesc tupleDescriptor);
@ -252,6 +254,10 @@ extern void SaveChunkGroups(RelFileNode relfilenode, uint64 stripe,
extern StripeSkipList * ReadStripeSkipList(RelFileNode relfilenode, uint64 stripe,
TupleDesc tupleDescriptor,
uint32 chunkCount);
extern StripeMetadata * FindStripeByRowNumber(Relation relation, uint64 rowNumber,
Snapshot snapshot);
extern StripeMetadata * FindStripeWithHighestRowNumber(Relation relation,
Snapshot snapshot);
extern Datum columnar_relation_storageid(PG_FUNCTION_ARGS);
@ -271,51 +277,5 @@ extern bool PendingWritesInUpperTransactions(Oid relfilenode,
SubTransactionId currentSubXid);
extern MemoryContext GetWriteContextForDebug(void);
typedef struct SmgrAddr
{
BlockNumber blockno;
uint32 offset;
} SmgrAddr;
/*
* Map logical offsets (as tracked in the metadata) to a physical page and
* offset where the data is kept.
*/
static inline SmgrAddr
logical_to_smgr(uint64 logicalOffset)
{
SmgrAddr addr;
addr.blockno = logicalOffset / COLUMNAR_BYTES_PER_PAGE;
addr.offset = SizeOfPageHeaderData + (logicalOffset % COLUMNAR_BYTES_PER_PAGE);
return addr;
}
/*
* Map a physical page adnd offset address to a logical address.
*/
static inline uint64
smgr_to_logical(SmgrAddr addr)
{
return COLUMNAR_BYTES_PER_PAGE * addr.blockno + addr.offset - SizeOfPageHeaderData;
}
/*
* Get the first usable address of next block.
*/
static inline SmgrAddr
next_block_start(SmgrAddr addr)
{
SmgrAddr result = {
.blockno = addr.blockno + 1,
.offset = SizeOfPageHeaderData
};
return result;
}
#endif /* COLUMNAR_H */

View File

@ -25,8 +25,10 @@ typedef struct StripeMetadata
uint32 chunkGroupRowCount;
uint64 rowCount;
uint64 id;
uint64 firstRowNumber;
} StripeMetadata;
extern List * StripesForRelfilenode(RelFileNode relfilenode);
extern void ColumnarStorageUpdateIfNeeded(Relation rel, bool isUpgrade);
#endif /* COLUMNAR_METADATA_H */

View File

@ -0,0 +1,64 @@
/*-------------------------------------------------------------------------
*
* columnar_storage.h
*
* Type and function declarations for storage of columnar data in blocks.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef COLUMNAR_STORAGE_H
#define COLUMNAR_STORAGE_H
#include "postgres.h"
#include "storage/smgr.h"
#include "utils/rel.h"
#include "columnar/columnar_tableam.h"
#define COLUMNAR_INVALID_ROW_NUMBER ((uint64) 0)
#define COLUMNAR_FIRST_ROW_NUMBER ((uint64) 1)
#define COLUMNAR_MAX_ROW_NUMBER ((uint64) \
(COLUMNAR_FIRST_ROW_NUMBER + \
VALID_ITEMPOINTER_OFFSETS * \
VALID_BLOCKNUMBERS))
/*
* Logical offsets never fall on the first two physical pages. See
* comments in columnar_storage.c.
*/
#define ColumnarInvalidLogicalOffset 0
#define ColumnarFirstLogicalOffset ((BLCKSZ - SizeOfPageHeaderData) * 2)
#define ColumnarLogicalOffsetIsValid(X) ((X) >= ColumnarFirstLogicalOffset)
extern void ColumnarStorageInit(SMgrRelation srel, uint64 storageId);
extern bool ColumnarStorageIsCurrent(Relation rel);
extern void ColumnarStorageUpdateCurrent(Relation rel, bool upgrade,
uint64 reservedStripeId,
uint64 reservedRowNumber,
uint64 reservedOffset);
extern uint64 ColumnarStorageGetVersionMajor(Relation rel, bool force);
extern uint64 ColumnarStorageGetVersionMinor(Relation rel, bool force);
extern uint64 ColumnarStorageGetStorageId(Relation rel, bool force);
extern uint64 ColumnarStorageGetReservedStripeId(Relation rel, bool force);
extern uint64 ColumnarStorageGetReservedRowNumber(Relation rel, bool force);
extern uint64 ColumnarStorageGetReservedOffset(Relation rel, bool force);
extern uint64 ColumnarStorageReserveData(Relation rel, uint64 amount);
extern uint64 ColumnarStorageReserveRowNumber(Relation rel, uint64 nrows);
extern uint64 ColumnarStorageReserveStripe(Relation rel);
extern void ColumnarStorageRead(Relation rel, uint64 logicalOffset,
char *data, uint32 amount);
extern void ColumnarStorageWrite(Relation rel, uint64 logicalOffset,
char *data, uint32 amount);
extern bool ColumnarStorageTruncate(Relation rel, uint64 newDataReservation);
#endif /* COLUMNAR_STORAGE_H */

View File

@ -8,6 +8,40 @@
#include "distributed/coordinator_protocol.h"
/*
* Number of valid ItemPointer Offset's for "row number" <> "ItemPointer"
* mapping.
*
* Postgres has some asserts calling either ItemPointerIsValid or
* OffsetNumberIsValid. That constraints itemPointer.offsetNumber
* for columnar tables to the following interval:
* [FirstOffsetNumber, MaxOffsetNumber].
*
* However, bitmap scan logic assumes that itemPointer.offsetNumber cannot
* be larger than MaxHeapTuplesPerPage (see tbm_add_tuples).
*
* For this reason, we restrict itemPointer.offsetNumber
* to the following interval: [FirstOffsetNumber, MaxHeapTuplesPerPage].
*/
#define VALID_ITEMPOINTER_OFFSETS \
((uint64) (MaxHeapTuplesPerPage - FirstOffsetNumber + 1))
/*
* Number of valid ItemPointer BlockNumber's for "row number" <> "ItemPointer"
* mapping.
*
* Similar to VALID_ITEMPOINTER_OFFSETS, due to asserts around
* itemPointer.blockNumber, we can only use values upto and including
* MaxBlockNumber.
* Note that postgres doesn't restrict blockNumber to a lower boundary.
*
* For this reason, we restrict itemPointer.blockNumber
* to the following interval: [0, MaxBlockNumber].
*/
#define VALID_BLOCKNUMBERS ((uint64) (MaxBlockNumber + 1))
const TableAmRoutine * GetColumnarTableAmRoutine(void);
extern void columnar_tableam_init(void);
extern void columnar_tableam_finish(void);

View File

@ -275,6 +275,7 @@ extern char * ChooseIndexName(const char *tabname, Oid namespaceId,
bool primary, bool isconstraint);
extern char * ChooseIndexNameAddition(List *colnames);
extern List * ChooseIndexColumnNames(List *indexElems);
extern LOCKMODE GetCreateIndexRelationLockMode(IndexStmt *createIndexStatement);
extern List * PreprocessReindexStmt(Node *ReindexStatement,
const char *ReindexCommand,
ProcessUtilityContext processUtilityContext);

View File

@ -39,10 +39,7 @@ typedef enum AdvisoryLocktagClass
ADV_LOCKTAG_CLASS_CITUS_REBALANCE_COLOCATION = 7,
ADV_LOCKTAG_CLASS_CITUS_COLOCATED_SHARDS_METADATA = 8,
ADV_LOCKTAG_CLASS_CITUS_OPERATIONS = 9,
ADV_LOCKTAG_CLASS_CITUS_PLACEMENT_CLEANUP = 10,
/* Columnar lock types */
ADV_LOCKTAG_CLASS_COLUMNAR_STRIPE_RESERVATION = 11
ADV_LOCKTAG_CLASS_CITUS_PLACEMENT_CLEANUP = 10
} AdvisoryLocktagClass;
/* CitusOperations has constants for citus operations */
@ -102,13 +99,6 @@ typedef enum CitusOperations
(uint32) operationId, \
ADV_LOCKTAG_CLASS_CITUS_OPERATIONS)
#define SET_LOCKTAG_COLUMNAR_STRIPE_RESERVATION(tag, relation) \
SET_LOCKTAG_ADVISORY(tag, \
relation->rd_lockInfo.lockRelId.dbId, \
relation->rd_lockInfo.lockRelId.relId, \
0, \
ADV_LOCKTAG_CLASS_COLUMNAR_STRIPE_RESERVATION)
/* reuse advisory lock, but with different, unused field 4 (10)
* Also it has the database hardcoded to MyDatabaseId, to ensure the locks
* are local to each database */

View File

@ -3,3 +3,4 @@
test: upgrade_basic_after
test: upgrade_partition_constraints_after
test: upgrade_pg_dist_object_test_after
test: upgrade_columnar_metapage_after

View File

@ -3,3 +3,4 @@
test: upgrade_basic_before
test: upgrade_partition_constraints_before
test: upgrade_pg_dist_object_test_before
test: upgrade_columnar_metapage_before

View File

@ -4,7 +4,7 @@ test: multi_test_catalog_views
test: columnar_create
test: columnar_load
test: columnar_query
test: columnar_query columnar_first_row_number
test: columnar_analyze
test: columnar_data_types
test: columnar_drop

View File

@ -12,6 +12,14 @@ WITH sample_data AS (VALUES
INSERT INTO test_alter_table SELECT * FROM sample_data;
-- drop a column
ALTER TABLE test_alter_table DROP COLUMN a;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_alter_table');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 2 | 150001 | 16402
(1 row)
-- test analyze
ANALYZE test_alter_table;
-- verify select queries run as expected
@ -59,6 +67,14 @@ SELECT * FROM test_alter_table;
3 | 5 | 8
(5 rows)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_alter_table');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 4 | 450001 | 32724
(1 row)
-- add a fixed-length column with default value
ALTER TABLE test_alter_table ADD COLUMN e int default 3;
SELECT * from test_alter_table;
@ -83,6 +99,14 @@ SELECT * from test_alter_table;
1 | 2 | 4 | 8
(6 rows)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_alter_table');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 5 | 600001 | 40906
(1 row)
-- add a variable-length column with default value
ALTER TABLE test_alter_table ADD COLUMN f text DEFAULT 'TEXT ME';
SELECT * from test_alter_table;
@ -231,7 +255,6 @@ insert into atacc1 values(1);
alter table atacc1
add column b float8 not null default random(),
add primary key(a);
ERROR: indexes not supported for columnar tables
-- Add a generate column with an expression value
create table test_gen_ex (x int) using columnar;
INSERT INTO test_gen_ex VALUES (1), (2), (3);
@ -366,30 +389,30 @@ SELECT * FROM products ORDER BY 1;
3 | pen | 2
(3 rows)
-- Add a UNIQUE constraint (should fail)
CREATE TABLE products_fail (
-- Add a UNIQUE constraint
CREATE TABLE products_unique (
product_no integer UNIQUE,
name text,
price numeric
) USING columnar;
ERROR: indexes not supported for columnar tables
ALTER TABLE products ADD COLUMN store_id text UNIQUE;
ERROR: indexes not supported for columnar tables
-- Add a PRIMARY KEY constraint (should fail)
CREATE TABLE products_fail (
-- Add a PRIMARY KEY constraint
CREATE TABLE products_primary (
product_no integer PRIMARY KEY,
name text,
price numeric
) USING columnar;
ERROR: indexes not supported for columnar tables
ALTER TABLE products ADD COLUMN store_id text PRIMARY KEY;
ERROR: indexes not supported for columnar tables
BEGIN;
ALTER TABLE products DROP COLUMN store_id;
ALTER TABLE products ADD COLUMN store_id text PRIMARY KEY;
ERROR: column "store_id" contains null values
ROLLBACK;
-- Add an EXCLUSION constraint (should fail)
CREATE TABLE circles (
c circle,
EXCLUDE USING gist (c WITH &&)
) USING columnar;
ERROR: indexes not supported for columnar tables
ERROR: only btree and hash indexes are supported on columnar tables
-- Row level security
CREATE TABLE public.row_level_security_col (id int, pgUser CHARACTER VARYING) USING columnar;
CREATE USER user1;

View File

@ -5,11 +5,14 @@
CREATE TABLE contestant (handle TEXT, birthdate DATE, rating INT,
percentile FLOAT, country CHAR(3), achievements TEXT[])
USING columnar;
-- should fail
SELECT alter_columnar_table_set('contestant', compression => 'none');
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
CREATE INDEX contestant_idx on contestant(handle);
ERROR: indexes not supported for columnar tables
-- Create compressed table with automatically determined file path
-- COMPRESSED
-- Create zstd compressed table
CREATE TABLE contestant_compressed (handle TEXT, birthdate DATE, rating INT,
percentile FLOAT, country CHAR(3), achievements TEXT[])
USING columnar;

View File

@ -52,6 +52,23 @@ select count(*) from t_compressed;
0
(1 row)
-- check storage
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_compressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_uncompressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
-- explain
explain (costs off, summary off, timing off) select * from t_uncompressed;
QUERY PLAN
@ -68,16 +85,16 @@ explain (costs off, summary off, timing off) select * from t_compressed;
-- vacuum
vacuum verbose t_compressed;
INFO: statistics for "t_compressed":
storage id: -1
total file size: 0, total data size: 0
storage id: xxxxx
total file size: 16384, total data size: 0
compression rate: 1.00x
total row count: 0, stripe count: 0, average rows per stripe: 0
chunk count: 0, containing data for dropped columns: 0
vacuum verbose t_uncompressed;
INFO: statistics for "t_uncompressed":
storage id: -1
total file size: 0, total data size: 0
storage id: xxxxx
total file size: 16384, total data size: 0
compression rate: 1.00x
total row count: 0, stripe count: 0, average rows per stripe: 0
chunk count: 0, containing data for dropped columns: 0
@ -85,6 +102,23 @@ chunk count: 0, containing data for dropped columns: 0
-- vacuum full
vacuum full t_compressed;
vacuum full t_uncompressed;
-- check storage
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_compressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_uncompressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
-- analyze
analyze t_uncompressed;
analyze t_compressed;
@ -94,6 +128,23 @@ truncate t_compressed;
-- alter type
alter table t_uncompressed alter column a type text;
alter table t_compressed alter column a type text;
-- check storage
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_compressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_uncompressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
-- verify cost of scanning an empty table is zero, not NaN
explain table t_uncompressed;
QUERY PLAN

View File

@ -0,0 +1,56 @@
CREATE SCHEMA columnar_first_row_number;
SET search_path tO columnar_first_row_number;
CREATE TABLE col_table_1 (a int) USING columnar;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 10) i;
BEGIN;
-- we don't use same first_row_number even if the xact is rollback'ed
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 11) i;
ROLLBACK;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 12) i;
SELECT alter_columnar_table_set('col_table_1', stripe_row_limit => 1000);
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 2350) i;
SELECT row_count, first_row_number FROM columnar.stripe a
WHERE a.storage_id = columnar_test_helpers.columnar_relation_storageid('col_table_1'::regclass)
ORDER BY stripe_num;
row_count | first_row_number
---------------------------------------------------------------------
10 | 1
12 | 300001
1000 | 450001
1000 | 451001
350 | 452001
(5 rows)
VACUUM FULL col_table_1;
-- show that we properly update first_row_number after VACUUM FULL
SELECT row_count, first_row_number FROM columnar.stripe a
WHERE a.storage_id = columnar_test_helpers.columnar_relation_storageid('col_table_1'::regclass)
ORDER BY stripe_num;
row_count | first_row_number
---------------------------------------------------------------------
1000 | 1
1000 | 1001
372 | 2001
(3 rows)
TRUNCATE col_table_1;
BEGIN;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 16) i;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 16) i;
COMMIT;
-- show that we start with first_row_number=1 after TRUNCATE
SELECT row_count, first_row_number FROM columnar.stripe a
WHERE a.storage_id = columnar_test_helpers.columnar_relation_storageid('col_table_1'::regclass)
ORDER BY stripe_num;
row_count | first_row_number
---------------------------------------------------------------------
32 | 1
(1 row)
SET client_min_messages TO ERROR;
DROP SCHEMA columnar_first_row_number CASCADE;

View File

@ -10,7 +10,7 @@ SET search_path tO columnar_indexes, public;
--
create table t(a int, b int) using columnar;
create index CONCURRENTLY t_idx on t(a, b);
ERROR: indexes not supported for columnar tables
ERROR: concurrent index commands are not supported for columnar tables
\d t
Table "columnar_indexes.t"
Column | Type | Collation | Nullable | Default
@ -32,16 +32,15 @@ SELECT * FROM t;
1 | 2
(1 row)
-- create index without the concurrent option. We should
-- error out during index creation.
create index t_idx on t(a, b);
ERROR: indexes not supported for columnar tables
\d t
Table "columnar_indexes.t"
Column | Type | Collation | Nullable | Default
---------------------------------------------------------------------
a | integer | | |
b | integer | | |
Indexes:
"t_idx" btree (a, b)
explain insert into t values (1, 2);
QUERY PLAN
@ -58,5 +57,347 @@ SELECT * FROM t;
3 | 4
(2 rows)
-- make sure that we test index scan
set columnar.enable_custom_scan to 'off';
set enable_seqscan to off;
CREATE table columnar_table (a INT, b int) USING columnar;
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(0, 16000) i;
-- unique --
BEGIN;
INSERT INTO columnar_table VALUES (100000000);
SAVEPOINT s1;
-- errors out due to unflushed data in upper transaction
CREATE UNIQUE INDEX ON columnar_table (a);
ERROR: cannot read from table when there is unflushed data in upper transactions
ROLLBACK;
CREATE UNIQUE INDEX ON columnar_table (a);
BEGIN;
INSERT INTO columnar_table VALUES (16050);
SAVEPOINT s1;
-- index scan errors out due to unflushed data in upper transaction
SELECT a FROM columnar_table WHERE a = 16050;
ERROR: cannot read from index when there is unflushed data in upper transactions
ROLLBACK;
EXPLAIN (COSTS OFF) SELECT * FROM columnar_table WHERE a=6456;
QUERY PLAN
---------------------------------------------------------------------
Index Scan using columnar_table_a_idx on columnar_table
Index Cond: (a = 6456)
(2 rows)
EXPLAIN (COSTS OFF) SELECT a FROM columnar_table WHERE a=6456;
QUERY PLAN
---------------------------------------------------------------------
Index Only Scan using columnar_table_a_idx on columnar_table
Index Cond: (a = 6456)
(2 rows)
SELECT (SELECT a FROM columnar_table WHERE a=6456 limit 1)=6456;
?column?
---------------------------------------------------------------------
t
(1 row)
SELECT (SELECT b FROM columnar_table WHERE a=6456 limit 1)=6456*2;
?column?
---------------------------------------------------------------------
t
(1 row)
-- even if a=16050 doesn't exist, we try to insert it twice so this should error out
INSERT INTO columnar_table VALUES (16050), (16050);
ERROR: duplicate key value violates unique constraint "columnar_table_a_idx"
DETAIL: Key (a)=(16050) already exists.
-- should work
INSERT INTO columnar_table VALUES (16050);
-- check edge cases around stripe boundaries, error out
INSERT INTO columnar_table VALUES (16050);
ERROR: duplicate key value violates unique constraint "columnar_table_a_idx"
DETAIL: Key (a)=(16050) already exists.
INSERT INTO columnar_table VALUES (15999);
ERROR: duplicate key value violates unique constraint "columnar_table_a_idx"
DETAIL: Key (a)=(15999) already exists.
DROP INDEX columnar_table_a_idx;
CREATE TABLE partial_unique_idx_test (a INT, b INT) USING columnar;
CREATE UNIQUE INDEX ON partial_unique_idx_test (a)
WHERE b > 500;
-- should work since b =< 500 and our partial index doesn't check this interval
INSERT INTO partial_unique_idx_test VALUES (1, 2), (1, 2);
-- should work since our partial index wouldn't cover the tuples that we inserted above
INSERT INTO partial_unique_idx_test VALUES (1, 800);
INSERT INTO partial_unique_idx_test VALUES (4, 600);
-- should error out due to (4, 600)
INSERT INTO partial_unique_idx_test VALUES (4, 700);
ERROR: duplicate key value violates unique constraint "partial_unique_idx_test_a_idx"
DETAIL: Key (a)=(4) already exists.
-- btree --
CREATE INDEX ON columnar_table (a);
SELECT (SELECT SUM(b) FROM columnar_table WHERE a>700 and a<965)=439560;
?column?
---------------------------------------------------------------------
t
(1 row)
CREATE INDEX ON columnar_table (b)
WHERE (b > 30000 AND b < 33000);
-- partial index should be way smaller than the non-partial index
SELECT pg_total_relation_size('columnar_table_b_idx') * 5 <
pg_total_relation_size('columnar_table_a_idx');
?column?
---------------------------------------------------------------------
t
(1 row)
-- can't use index scan due to partial index boundaries
EXPLAIN (COSTS OFF) SELECT b FROM columnar_table WHERE b = 30000;
QUERY PLAN
---------------------------------------------------------------------
Seq Scan on columnar_table
Filter: (b = 30000)
(2 rows)
-- can use index scan
EXPLAIN (COSTS OFF) SELECT b FROM columnar_table WHERE b = 30001;
QUERY PLAN
---------------------------------------------------------------------
Index Only Scan using columnar_table_b_idx on columnar_table
Index Cond: (b = 30001)
(2 rows)
-- some more rows
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(16000, 17000) i;
DROP INDEX columnar_table_a_idx;
TRUNCATE columnar_table;
-- pkey --
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(16000, 16499) i;
ALTER TABLE columnar_table ADD PRIMARY KEY (a);
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(16500, 17000) i;
BEGIN;
INSERT INTO columnar_table (a) SELECT 1;
ROLLBACK;
-- should work
INSERT INTO columnar_table (a) SELECT 1;
-- error out
INSERT INTO columnar_table VALUES (16100), (16101);
ERROR: duplicate key value violates unique constraint "columnar_table_pkey"
DETAIL: Key (a)=(16100) already exists.
INSERT INTO columnar_table VALUES (16999);
ERROR: duplicate key value violates unique constraint "columnar_table_pkey"
DETAIL: Key (a)=(16999) already exists.
BEGIN;
REINDEX INDEX columnar_table_pkey;
-- should error even after reindex
INSERT INTO columnar_table VALUES (16999);
ERROR: duplicate key value violates unique constraint "columnar_table_pkey"
DETAIL: Key (a)=(16999) already exists.
ROLLBACK;
VACUUM FULL columnar_table;
-- show that we don't support clustering columnar tables using indexes
CLUSTER columnar_table USING columnar_table_pkey;
ERROR: clustering columnar tables using indexes is not supported
ALTER TABLE columnar_table CLUSTER ON columnar_table_pkey;
CLUSTER columnar_table;
ERROR: clustering columnar tables using indexes is not supported
-- should error even after vacuum
INSERT INTO columnar_table VALUES (16999);
ERROR: duplicate key value violates unique constraint "columnar_table_pkey"
DETAIL: Key (a)=(16999) already exists.
TRUNCATE columnar_table;
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(1, 160000) i;
SELECT (SELECT b FROM columnar_table WHERE a = 150000)=300000;
?column?
---------------------------------------------------------------------
t
(1 row)
TRUNCATE columnar_table;
ALTER TABLE columnar_table DROP CONSTRAINT columnar_table_pkey;
-- hash --
INSERT INTO columnar_table (a, b) SELECT i*2,i FROM generate_series(1, 8000) i;
CREATE INDEX hash_idx ON columnar_table USING HASH (b);
BEGIN;
CREATE INDEX hash_idx_fill_factor ON columnar_table USING HASH (b) WITH (fillfactor=10);
-- same hash index with lower fillfactor should be way bigger
SELECT pg_total_relation_size ('hash_idx_fill_factor') >
pg_total_relation_size ('hash_idx') * 5;
?column?
---------------------------------------------------------------------
t
(1 row)
ROLLBACK;
BEGIN;
INSERT INTO columnar_table (a, b) SELECT i*3,i FROM generate_series(1, 8000) i;
ROLLBACK;
INSERT INTO columnar_table (a, b) SELECT i*4,i FROM generate_series(1, 8000) i;
SELECT SUM(a)=42000 FROM columnar_table WHERE b = 7000;
?column?
---------------------------------------------------------------------
t
(1 row)
BEGIN;
REINDEX TABLE columnar_table;
SELECT SUM(a)=42000 FROM columnar_table WHERE b = 7000;
?column?
---------------------------------------------------------------------
t
(1 row)
ROLLBACK;
VACUUM FULL columnar_table;
SELECT SUM(a)=42000 FROM columnar_table WHERE b = 7000;
?column?
---------------------------------------------------------------------
t
(1 row)
-- exclusion contraints --
CREATE TABLE exclusion_test (c1 INT,c2 INT, c3 INT, c4 BOX,
EXCLUDE USING btree (c1 WITH =) INCLUDE(c3,c4) WHERE (c1 < 10)) USING columnar;
-- error out since "c1" is "1" for all rows to be inserted
INSERT INTO exclusion_test SELECT 1, 2, 3*x, BOX('4,4,4,4') FROM generate_series(1,3) AS x;
ERROR: conflicting key value violates exclusion constraint "exclusion_test_c1_c3_c4_excl"
DETAIL: Key (c1)=(1) conflicts with existing key (c1)=(1).
BEGIN;
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(1,3) AS x;
ROLLBACK;
-- should work
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(1,3) AS x;
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(10,15) AS x;
BEGIN;
-- should work thanks to "where" clause in exclusion constraint
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(10,15) AS x;
ROLLBACK;
REINDEX TABLE exclusion_test;
-- should still work after reindex
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(10,15) AS x;
-- make sure that we respect INCLUDE syntax --
CREATE TABLE include_test (a INT, b BIGINT, c BIGINT, d BIGINT) USING columnar;
INSERT INTO include_test SELECT i, i, i, i FROM generate_series (1, 1000) i;
CREATE UNIQUE INDEX unique_a ON include_test (a);
-- cannot use index only scan
EXPLAIN (COSTS OFF) SELECT b FROM include_test WHERE a = 500;
QUERY PLAN
---------------------------------------------------------------------
Index Scan using unique_a on include_test
Index Cond: (a = 500)
(2 rows)
CREATE UNIQUE INDEX unique_a_include_b_c_d ON include_test (a) INCLUDE(b, c, d);
-- same unique index that includes other columns should be way bigger
SELECT pg_total_relation_size ('unique_a') * 1.5 <
pg_total_relation_size ('unique_a_include_b_c_d');
?column?
---------------------------------------------------------------------
t
(1 row)
DROP INDEX unique_a;
-- should use index only scan since unique_a_include_b_c_d includes column "b" too
EXPLAIN (COSTS OFF) SELECT b FROM include_test WHERE a = 500;
QUERY PLAN
---------------------------------------------------------------------
Index Only Scan using unique_a_include_b_c_d on include_test
Index Cond: (a = 500)
(2 rows)
BEGIN;
SET enable_indexonlyscan = OFF;
-- show that we respect enable_indexonlyscan GUC
EXPLAIN (COSTS OFF) SELECT b FROM include_test WHERE a = 500;
QUERY PLAN
---------------------------------------------------------------------
Index Scan using unique_a_include_b_c_d on include_test
Index Cond: (a = 500)
(2 rows)
ROLLBACK;
-- make sure that we read the correct value for "b" when doing index only scan
SELECT b=980 FROM include_test WHERE a = 980;
?column?
---------------------------------------------------------------------
t
(1 row)
-- some tests with distributed & partitioned tables --
CREATE TABLE dist_part_table(
dist_col INT,
part_col TIMESTAMPTZ,
col1 TEXT
) PARTITION BY RANGE (part_col);
-- create an index before creating a columnar partition
CREATE INDEX dist_part_table_btree ON dist_part_table (col1);
-- columnar partition
CREATE TABLE p0 PARTITION OF dist_part_table
FOR VALUES FROM ('2020-01-01') TO ('2020-02-01')
USING columnar;
SELECT create_distributed_table('dist_part_table', 'dist_col');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- columnar partition
CREATE TABLE p1 PARTITION OF dist_part_table
FOR VALUES FROM ('2020-02-01') TO ('2020-03-01')
USING columnar;
-- row partition
CREATE TABLE p2 PARTITION OF dist_part_table
FOR VALUES FROM ('2020-03-01') TO ('2020-04-01');
INSERT INTO dist_part_table VALUES (1, '2020-03-15', 'str1', POINT(1, 1));
ERROR: INSERT has more expressions than target columns
-- insert into columnar partitions
INSERT INTO dist_part_table VALUES (1, '2020-01-15', 'str2', POINT(2, 2));
ERROR: INSERT has more expressions than target columns
INSERT INTO dist_part_table VALUES (1, '2020-02-15', 'str3', POINT(3, 3));
ERROR: INSERT has more expressions than target columns
-- create another index after creating a columnar partition
CREATE UNIQUE INDEX dist_part_table_unique ON dist_part_table (dist_col, part_col);
-- verify that indexes are created on columnar partitions
SELECT COUNT(*)=2 FROM pg_indexes WHERE tablename = 'p0';
?column?
---------------------------------------------------------------------
t
(1 row)
SELECT COUNT(*)=2 FROM pg_indexes WHERE tablename = 'p1';
?column?
---------------------------------------------------------------------
t
(1 row)
-- unsupported index types --
-- gin --
CREATE TABLE testjsonb (j JSONB) USING columnar;
INSERT INTO testjsonb SELECT CAST('{"f1" : ' ||'"'|| i*4 ||'", ' || '"f2" : '||'"'|| i*10 ||'"}' AS JSON) FROM generate_series(1,10) i;
CREATE INDEX jidx ON testjsonb USING GIN (j);
ERROR: only btree and hash indexes are supported on columnar tables
INSERT INTO testjsonb SELECT CAST('{"f1" : ' ||'"'|| i*4 ||'", ' || '"f2" : '||'"'|| i*10 ||'"}' AS JSON) FROM generate_series(15,20) i;
-- gist --
CREATE TABLE gist_point_tbl(id INT4, p POINT) USING columnar;
INSERT INTO gist_point_tbl (id, p) SELECT g, point(g*10, g*10) FROM generate_series(1, 10) g;
CREATE INDEX gist_pointidx ON gist_point_tbl USING gist(p);
ERROR: only btree and hash indexes are supported on columnar tables
INSERT INTO gist_point_tbl (id, p) SELECT g, point(g*10, g*10) FROM generate_series(10, 20) g;
-- sp gist --
CREATE TABLE box_temp (f1 box) USING columnar;
INSERT INTO box_temp SELECT box(point(i, i), point(i * 2, i * 2)) FROM generate_series(1, 10) AS i;
CREATE INDEX box_spgist ON box_temp USING spgist (f1);
ERROR: only btree and hash indexes are supported on columnar tables
INSERT INTO box_temp SELECT box(point(i, i), point(i * 2, i * 2)) FROM generate_series(1, 10) AS i;
-- brin --
CREATE TABLE brin_summarize (value int) USING columnar;
CREATE INDEX brin_summarize_idx ON brin_summarize USING brin (value) WITH (pages_per_range=2);
ERROR: only btree and hash indexes are supported on columnar tables
-- Show that we safely fallback to serial index build.
CREATE TABLE parallel_scan_test(a int) USING columnar WITH ( parallel_workers = 2 );
INSERT INTO parallel_scan_test SELECT i FROM generate_series(1,10) i;
CREATE INDEX ON parallel_scan_test (a);
NOTICE: falling back to serial index build since parallel scan on columnar tables is not supported
VACUUM FULL parallel_scan_test;
NOTICE: falling back to serial index build since parallel scan on columnar tables is not supported
REINDEX TABLE parallel_scan_test;
NOTICE: falling back to serial index build since parallel scan on columnar tables is not supported
SET client_min_messages TO WARNING;
DROP SCHEMA columnar_indexes CASCADE;

View File

@ -45,6 +45,14 @@ select count(*) from test_insert_command;
3
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_insert_command');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 4 | 450001 | 32686
(1 row)
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
consistent
---------------------------------------------------------------------
@ -141,6 +149,14 @@ FROM test_toast_columnar;
5004 | 5004 | 5004 | 5004
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_toast_columnar');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 2 | 150001 | 16428
(1 row)
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
consistent
---------------------------------------------------------------------
@ -173,6 +189,14 @@ INSERT INTO zero_col_heap SELECT * FROM zero_col_heap;
INSERT INTO zero_col_heap SELECT * FROM zero_col_heap;
INSERT INTO zero_col_heap SELECT * FROM zero_col_heap;
INSERT INTO zero_col SELECT * FROM zero_col_heap;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('zero_col');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 6 | 750001 | 16336
(1 row)
SELECT relname, stripe_num, chunk_group_count, row_count FROM columnar.stripe a, pg_class b
WHERE columnar_test_helpers.columnar_relation_storageid(b.oid)=a.storage_id AND relname = 'zero_col'
ORDER BY 1,2,3,4;

View File

@ -14,6 +14,14 @@ SELECT count(*) FROM t;
0
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 150001 | 16336
(1 row)
-- check stripe metadata also have been rolled-back
SELECT count(*) FROM t_stripes;
count
@ -46,6 +54,14 @@ SELECT count(*) FROM t; -- force flush
SAVEPOINT s1;
INSERT INTO t SELECT i, i+1 FROM generate_series(1, 10) i;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 3 | 600001 | 24606
(1 row)
SELECT count(*) FROM t;
count
---------------------------------------------------------------------
@ -68,6 +84,14 @@ SELECT count(*) FROM t;
INSERT INTO t SELECT i, i+1 FROM generate_series(1, 10) i;
COMMIT;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 5 | 750001 | 40942
(1 row)
SELECT count(*) FROM t;
count
---------------------------------------------------------------------

View File

@ -3,6 +3,16 @@ SET search_path TO columnar_test_helpers;
CREATE FUNCTION columnar_relation_storageid(relid oid) RETURNS bigint
LANGUAGE C STABLE STRICT
AS 'citus', $$columnar_relation_storageid$$;
CREATE OR REPLACE FUNCTION columnar_storage_info(
rel regclass,
version_major OUT int4,
version_minor OUT int4,
storage_id OUT int8,
reserved_stripe_id OUT int8,
reserved_row_number OUT int8,
reserved_offset OUT int8)
STRICT
LANGUAGE c AS 'citus', $$columnar_storage_info$$;
CREATE FUNCTION compression_type_supported(type text) RETURNS boolean
AS $$
BEGIN

View File

@ -43,7 +43,23 @@ SELECT * FROM columnar_test_helpers.chunk_group_consistency;
t
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('columnar_truncate_test');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 2 | 150001 | 16438
(1 row)
TRUNCATE TABLE columnar_truncate_test;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('columnar_truncate_test');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 1 | 1 | 16336
(1 row)
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
consistent
---------------------------------------------------------------------
@ -77,7 +93,7 @@ SELECT count(*) FROM columnar_truncate_test_compressed;
SELECT pg_relation_size('columnar_truncate_test_compressed');
pg_relation_size
---------------------------------------------------------------------
0
16384
(1 row)
INSERT INTO columnar_truncate_test select a, a from generate_series(1, 10) a;

View File

@ -25,6 +25,14 @@ SELECT count(*) FROM t_stripes;
3
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 4 | 450001 | 32756
(1 row)
-- vacuum full should merge stripes together
VACUUM FULL t;
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
@ -45,6 +53,14 @@ SELECT count(*) FROM t_stripes;
1
(1 row)
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 2 | 150001 | 16584
(1 row)
-- test the case when all data cannot fit into a single stripe
SELECT alter_columnar_table_set('t', stripe_row_limit => 1000);
alter_columnar_table_set
@ -66,6 +82,14 @@ SELECT count(*) FROM t_stripes;
(1 row)
VACUUM FULL t;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 4 | 3001 | 53382
(1 row)
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
consistent
---------------------------------------------------------------------
@ -215,6 +239,14 @@ compression rate: 1.25x
total row count: 5530, stripe count: 5, average rows per stripe: 1106
chunk count: 7, containing data for dropped columns: 0, none compressed: 5, pglz compressed: 2
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 16 | 21001 | 50686
(1 row)
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
consistent
---------------------------------------------------------------------

View File

@ -140,3 +140,45 @@ a b
11
12
13
starting permutation: s1-truncate s1-begin s1-insert-10000-rows s2-begin s2-insert s2-commit s1-commit s1-verify-metadata
step s1-truncate:
TRUNCATE test_insert_concurrency;
step s1-begin:
BEGIN;
step s1-insert-10000-rows:
INSERT INTO test_insert_concurrency SELECT i, 2 * i FROM generate_series(1, 10000) i;
step s2-begin:
BEGIN;
step s2-insert:
INSERT INTO test_insert_concurrency SELECT i, 2 * i FROM generate_series(4, 6) i;
step s2-commit:
COMMIT;
step s1-commit:
COMMIT;
step s1-verify-metadata:
WITH test_insert_concurrency_stripes AS (
SELECT first_row_number, stripe_num, row_count
FROM columnar.stripe a, pg_class b
WHERE columnar_relation_storageid(b.oid)=a.storage_id AND
relname = 'test_insert_concurrency'
)
SELECT
-- verify that table has two stripes ..
count(*) = 2 AND
-- .. and those stripes look like:
sum(case when stripe_num = 1 AND first_row_number = 150001 AND row_count = 3 then 1 end) = 1 AND
sum(case when stripe_num = 2 AND first_row_number = 1 AND row_count = 10000 then 1 end) = 1
AS stripe_metadata_for_test_insert_concurrency_ok
FROM test_insert_concurrency_stripes;
stripe_metadata_for_test_insert_concurrency_ok
t

View File

@ -595,9 +595,11 @@ SELECT * FROM print_extension_changes();
-- Snapshot of state at 10.2-1
ALTER EXTENSION citus UPDATE TO '10.2-1';
SELECT * FROM print_extension_changes();
previous_object | current_object
previous_object | current_object
---------------------------------------------------------------------
(0 rows)
| function citus_internal.downgrade_columnar_storage(regclass) void
| function citus_internal.upgrade_columnar_storage(regclass) void
(2 rows)
DROP TABLE prev_objects, extension_diff;
-- show running version

View File

@ -259,7 +259,7 @@ ABORT;
-- all below 5 commands should throw no permission errors
-- read columnar metadata table
SELECT * FROM columnar.stripe;
storage_id | stripe_num | file_offset | data_length | column_count | chunk_row_count | row_count | chunk_group_count
storage_id | stripe_num | file_offset | data_length | column_count | chunk_row_count | row_count | chunk_group_count | first_row_number
---------------------------------------------------------------------
(0 rows)

View File

@ -0,0 +1,122 @@
\set upgrade_test_old_citus_version `echo "$upgrade_test_old_citus_version"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int >= 10 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int >= 0
AS upgrade_test_old_citus_version_ge_10_0;
upgrade_test_old_citus_version_ge_10_0
---------------------------------------------------------------------
t
(1 row)
\gset
\if :upgrade_test_old_citus_version_ge_10_0
\else
\q
\endif
-- it's not the best practice to define this here, but we don't want to include
-- columnar_test_helpers in upgrade test schedule
CREATE OR REPLACE FUNCTION columnar_storage_info(
rel regclass,
version_major OUT int4,
version_minor OUT int4,
storage_id OUT int8,
reserved_stripe_id OUT int8,
reserved_row_number OUT int8,
reserved_offset OUT int8)
STRICT
LANGUAGE c AS 'citus', 'columnar_storage_info';
CREATE VIEW columnar_table_stripe_info AS
SELECT columnar_table_storageids.relname relname,
columnar.stripe.stripe_num stripe_num,
columnar.stripe.row_count row_count,
columnar.stripe.first_row_number first_row_number
FROM columnar.stripe,
(
SELECT c.oid relid, c.relname relname, (columnar_storage_info(c.oid)).storage_id relstorageid
FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar'
) columnar_table_storageids
WHERE relstorageid = columnar.stripe.storage_id;
SET search_path TO upgrade_columnar_metapage, public;
-- show that first_row_number values are equal to MAX(row_count) * stripe_num + COLUMNAR_FIRST_ROW_NUMBER
SELECT * FROM columnar_table_stripe_info ORDER BY relname, stripe_num;
relname | stripe_num | row_count | first_row_number
---------------------------------------------------------------------
columnar_table_1 | 1 | 150000 | 1
columnar_table_1 | 2 | 10000 | 150001
columnar_table_2 | 1 | 1000 | 1
columnar_table_2 | 2 | 901 | 150001
columnar_table_3 | 1 | 2 | 1
(5 rows)
-- should work since we upgrade metapages when upgrading schema version
INSERT INTO columnar_table_1 VALUES (3);
-- state of stripe metadata for columnar_table_1 after post-upgrade insert
SELECT * FROM columnar_table_stripe_info WHERE relname = 'columnar_table_1' ORDER BY stripe_num;
relname | stripe_num | row_count | first_row_number
---------------------------------------------------------------------
columnar_table_1 | 1 | 150000 | 1
columnar_table_1 | 2 | 10000 | 150001
columnar_table_1 | 3 | 1 | 160001
(3 rows)
-- show that all columnar relation's metapage's are upgraded to "2.0"
SELECT count(*)=0
FROM (SELECT (columnar_storage_info(c.oid)).* t
FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar') t
WHERE t.version_major != 2 and t.version_minor != 0;
?column?
---------------------------------------------------------------------
t
(1 row)
-- print metapage for two of the tables
SELECT columnar_storage_info('columnar_table_1');
columnar_storage_info
---------------------------------------------------------------------
(2,0,10000000000,4,310001,481936)
(1 row)
SELECT columnar_storage_info('columnar_table_2');
columnar_storage_info
---------------------------------------------------------------------
(2,0,10000000001,3,150902,26694)
(1 row)
-- show that no_data_columnar_table also has metapage after upgrade
SELECT columnar_storage_info('no_data_columnar_table');
columnar_storage_info
---------------------------------------------------------------------
(2,0,10000000003,1,1,16336)
(1 row)
-- table is already upgraded, make sure that upgrade_columnar_metapage is no-op
SELECT citus_internal.upgrade_columnar_storage(c.oid)
FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar' and relname = 'columnar_table_2';
upgrade_columnar_storage
---------------------------------------------------------------------
(1 row)
SELECT columnar_storage_info('columnar_table_2');
columnar_storage_info
---------------------------------------------------------------------
(2,0,10000000001,3,150902,26694)
(1 row)
VACUUM FULL columnar_table_2;
-- print metapage and stripe metadata after post-upgrade vacuum full
SELECT columnar_storage_info('columnar_table_2');
columnar_storage_info
---------------------------------------------------------------------
(2,0,10000000004,3,2001,26694)
(1 row)
SELECT * FROM columnar_table_stripe_info WHERE relname = 'columnar_table_2' ORDER BY stripe_num;
relname | stripe_num | row_count | first_row_number
---------------------------------------------------------------------
columnar_table_2 | 1 | 1000 | 1
columnar_table_2 | 2 | 901 | 1001
(2 rows)

View File

@ -0,0 +1,13 @@
\set upgrade_test_old_citus_version `echo "$upgrade_test_old_citus_version"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int >= 10 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int >= 0
AS upgrade_test_old_citus_version_ge_10_0;
upgrade_test_old_citus_version_ge_10_0
---------------------------------------------------------------------
f
(1 row)
\gset
\if :upgrade_test_old_citus_version_ge_10_0
\else
\q

View File

@ -0,0 +1,31 @@
\set upgrade_test_old_citus_version `echo "$upgrade_test_old_citus_version"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int >= 10 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int >= 0
AS upgrade_test_old_citus_version_ge_10_0;
upgrade_test_old_citus_version_ge_10_0
---------------------------------------------------------------------
t
(1 row)
\gset
\if :upgrade_test_old_citus_version_ge_10_0
\else
\q
\endif
CREATE SCHEMA upgrade_columnar_metapage;
SET search_path TO upgrade_columnar_metapage, public;
CREATE TABLE columnar_table_1(a INT, b INT) USING columnar;
INSERT INTO columnar_table_1 SELECT i FROM generate_series(160001, 320000) i;
CREATE TABLE columnar_table_2(b INT) USING columnar;
SELECT alter_columnar_table_set('columnar_table_2',
chunk_group_row_limit => 100,
stripe_row_limit => 1000);
alter_columnar_table_set
---------------------------------------------------------------------
(1 row)
INSERT INTO columnar_table_2 SELECT i FROM generate_series(1600, 3500) i;
CREATE TABLE columnar_table_3(b INT) USING columnar;
INSERT INTO columnar_table_3 VALUES (1), (2);
CREATE TABLE no_data_columnar_table(a INT, b INT, c TEXT) USING columnar;

View File

@ -0,0 +1,13 @@
\set upgrade_test_old_citus_version `echo "$upgrade_test_old_citus_version"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int >= 10 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int >= 0
AS upgrade_test_old_citus_version_ge_10_0;
upgrade_test_old_citus_version_ge_10_0
---------------------------------------------------------------------
f
(1 row)
\gset
\if :upgrade_test_old_citus_version_ge_10_0
\else
\q

View File

@ -57,6 +57,7 @@ ORDER BY 1;
function citus_extradata_container(internal)
function citus_finish_pg_upgrade()
function citus_get_active_worker_nodes()
function citus_internal.downgrade_columnar_storage(regclass)
function citus_internal.find_groupid_for_node(text,integer)
function citus_internal.pg_dist_node_trigger_func()
function citus_internal.pg_dist_rebalance_strategy_trigger_func()
@ -64,6 +65,7 @@ ORDER BY 1;
function citus_internal.refresh_isolation_tester_prepared_statement()
function citus_internal.replace_isolation_tester_func()
function citus_internal.restore_isolation_tester_func()
function citus_internal.upgrade_columnar_storage(regclass)
function citus_isolation_test_session_is_blocked(integer,integer[])
function citus_json_concatenate(json,json)
function citus_json_concatenate_final(json)
@ -246,5 +248,5 @@ ORDER BY 1;
view citus_worker_stat_activity
view pg_dist_shard_placement
view time_partitions
(230 rows)
(232 rows)

View File

@ -53,6 +53,7 @@ ORDER BY 1;
function citus_extradata_container(internal)
function citus_finish_pg_upgrade()
function citus_get_active_worker_nodes()
function citus_internal.downgrade_columnar_storage(regclass)
function citus_internal.find_groupid_for_node(text,integer)
function citus_internal.pg_dist_node_trigger_func()
function citus_internal.pg_dist_rebalance_strategy_enterprise_check()
@ -61,6 +62,7 @@ ORDER BY 1;
function citus_internal.refresh_isolation_tester_prepared_statement()
function citus_internal.replace_isolation_tester_func()
function citus_internal.restore_isolation_tester_func()
function citus_internal.upgrade_columnar_storage(regclass)
function citus_isolation_test_session_is_blocked(integer,integer[])
function citus_json_concatenate(json,json)
function citus_json_concatenate_final(json)
@ -238,5 +240,5 @@ ORDER BY 1;
view citus_worker_stat_activity
view pg_dist_shard_placement
view time_partitions
(222 rows)
(224 rows)

View File

@ -15,14 +15,20 @@ COPY contestant FROM '@abs_srcdir@/data/contestants.1.csv' WITH CSV;
-- COPY into uncompressed table from program
COPY contestant FROM PROGRAM 'cat @abs_srcdir@/data/contestants.2.csv' WITH CSV;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('contestant');
-- COPY into compressed table
set columnar.compression = 'pglz';
COPY contestant_compressed FROM '@abs_srcdir@/data/contestants.1.csv' WITH CSV;
-- COPY into uncompressed table from program
COPY contestant_compressed FROM PROGRAM 'cat @abs_srcdir@/data/contestants.2.csv'
WITH CSV;
set columnar.compression to default;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('contestant_compressed');
-- Test column list
CREATE TABLE famous_constants (id int, name text, value real)

View File

@ -14,13 +14,27 @@ DETAIL: command not found
COPY contestant FROM '@abs_srcdir@/data/contestants.1.csv' WITH CSV;
-- COPY into uncompressed table from program
COPY contestant FROM PROGRAM 'cat @abs_srcdir@/data/contestants.2.csv' WITH CSV;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('contestant');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 3 | 300001 | 24742
(1 row)
-- COPY into compressed table
set columnar.compression = 'pglz';
COPY contestant_compressed FROM '@abs_srcdir@/data/contestants.1.csv' WITH CSV;
-- COPY into uncompressed table from program
COPY contestant_compressed FROM PROGRAM 'cat @abs_srcdir@/data/contestants.2.csv'
WITH CSV;
set columnar.compression to default;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('contestant_compressed');
version_major | version_minor | reserved_stripe_id | reserved_row_number | reserved_offset
---------------------------------------------------------------------
2 | 0 | 3 | 300001 | 24704
(1 row)
-- Test column list
CREATE TABLE famous_constants (id int, name text, value real)
USING columnar;

View File

@ -1,6 +1,10 @@
setup
{
CREATE TABLE test_insert_concurrency (a int, b int) USING columnar;
CREATE OR REPLACE FUNCTION columnar_relation_storageid(relid oid) RETURNS bigint
LANGUAGE C STABLE STRICT
AS 'citus', $$columnar_relation_storageid$$;
}
teardown
@ -20,6 +24,11 @@ step "s1-insert"
INSERT INTO test_insert_concurrency SELECT i, 2 * i FROM generate_series(1, 3) i;
}
step "s1-insert-10000-rows"
{
INSERT INTO test_insert_concurrency SELECT i, 2 * i FROM generate_series(1, 10000) i;
}
step "s1-copy"
{
COPY test_insert_concurrency(a) FROM PROGRAM 'seq 11 13';
@ -30,6 +39,29 @@ step "s1-select"
SELECT * FROM test_insert_concurrency ORDER BY a;
}
step "s1-truncate"
{
TRUNCATE test_insert_concurrency;
}
step "s1-verify-metadata"
{
WITH test_insert_concurrency_stripes AS (
SELECT first_row_number, stripe_num, row_count
FROM columnar.stripe a, pg_class b
WHERE columnar_relation_storageid(b.oid)=a.storage_id AND
relname = 'test_insert_concurrency'
)
SELECT
-- verify that table has two stripes ..
count(*) = 2 AND
-- .. and those stripes look like:
sum(case when stripe_num = 1 AND first_row_number = 150001 AND row_count = 3 then 1 end) = 1 AND
sum(case when stripe_num = 2 AND first_row_number = 1 AND row_count = 10000 then 1 end) = 1
AS stripe_metadata_for_test_insert_concurrency_ok
FROM test_insert_concurrency_stripes;
}
step "s1-commit"
{
COMMIT;
@ -65,3 +97,9 @@ permutation "s1-begin" "s2-begin" "s1-copy" "s2-insert" "s1-select" "s2-select"
// insert vs copy
permutation "s1-begin" "s2-begin" "s2-insert" "s1-copy" "s1-select" "s2-select" "s1-commit" "s2-commit" "s1-select"
# insert vs insert
# Start inserting rows in session 1, reserve first_row_number to be 1 for session 1 but commit session 2 before session 1.
# Then verify that while the stripe written by session 2 has the greater first_row_number, stripe written by session 1 has
# the greater stripe_num. This is because, we reserve stripe_num and first_row_number at different times.
permutation "s1-truncate" "s1-begin" "s1-insert-10000-rows" "s2-begin" "s2-insert" "s2-commit" "s1-commit" "s1-verify-metadata"

View File

@ -17,6 +17,10 @@ INSERT INTO test_alter_table SELECT * FROM sample_data;
-- drop a column
ALTER TABLE test_alter_table DROP COLUMN a;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_alter_table');
-- test analyze
ANALYZE test_alter_table;
@ -36,6 +40,10 @@ SELECT * FROM test_alter_table;
INSERT INTO test_alter_table (SELECT 3, 5, 8);
SELECT * FROM test_alter_table;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_alter_table');
-- add a fixed-length column with default value
ALTER TABLE test_alter_table ADD COLUMN e int default 3;
@ -43,6 +51,10 @@ SELECT * from test_alter_table;
INSERT INTO test_alter_table (SELECT 1, 2, 4, 8);
SELECT * from test_alter_table;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_alter_table');
-- add a variable-length column with default value
ALTER TABLE test_alter_table ADD COLUMN f text DEFAULT 'TEXT ME';
@ -203,21 +215,25 @@ ALTER TABLE products DROP CONSTRAINT dummy_constraint;
INSERT INTO products VALUES (3, 'pen', 2);
SELECT * FROM products ORDER BY 1;
-- Add a UNIQUE constraint (should fail)
CREATE TABLE products_fail (
-- Add a UNIQUE constraint
CREATE TABLE products_unique (
product_no integer UNIQUE,
name text,
price numeric
) USING columnar;
ALTER TABLE products ADD COLUMN store_id text UNIQUE;
-- Add a PRIMARY KEY constraint (should fail)
CREATE TABLE products_fail (
-- Add a PRIMARY KEY constraint
CREATE TABLE products_primary (
product_no integer PRIMARY KEY,
name text,
price numeric
) USING columnar;
ALTER TABLE products ADD COLUMN store_id text PRIMARY KEY;
BEGIN;
ALTER TABLE products DROP COLUMN store_id;
ALTER TABLE products ADD COLUMN store_id text PRIMARY KEY;
ROLLBACK;
-- Add an EXCLUSION constraint (should fail)
CREATE TABLE circles (

View File

@ -7,12 +7,11 @@
CREATE TABLE contestant (handle TEXT, birthdate DATE, rating INT,
percentile FLOAT, country CHAR(3), achievements TEXT[])
USING columnar;
SELECT alter_columnar_table_set('contestant', compression => 'none');
-- should fail
CREATE INDEX contestant_idx on contestant(handle);
-- Create compressed table with automatically determined file path
-- COMPRESSED
-- Create zstd compressed table
CREATE TABLE contestant_compressed (handle TEXT, birthdate DATE, rating INT,
percentile FLOAT, country CHAR(3), achievements TEXT[])
USING columnar;

View File

@ -19,6 +19,14 @@ select count(*) from t_uncompressed;
select * from t_compressed;
select count(*) from t_compressed;
-- check storage
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_compressed');
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_uncompressed');
-- explain
explain (costs off, summary off, timing off) select * from t_uncompressed;
explain (costs off, summary off, timing off) select * from t_compressed;
@ -31,6 +39,14 @@ vacuum verbose t_uncompressed;
vacuum full t_compressed;
vacuum full t_uncompressed;
-- check storage
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_compressed');
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_uncompressed');
-- analyze
analyze t_uncompressed;
analyze t_compressed;
@ -43,6 +59,14 @@ truncate t_compressed;
alter table t_uncompressed alter column a type text;
alter table t_compressed alter column a type text;
-- check storage
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_compressed');
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t_uncompressed');
-- verify cost of scanning an empty table is zero, not NaN
explain table t_uncompressed;
explain table t_compressed;

View File

@ -0,0 +1,43 @@
CREATE SCHEMA columnar_first_row_number;
SET search_path tO columnar_first_row_number;
CREATE TABLE col_table_1 (a int) USING columnar;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 10) i;
BEGIN;
-- we don't use same first_row_number even if the xact is rollback'ed
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 11) i;
ROLLBACK;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 12) i;
SELECT alter_columnar_table_set('col_table_1', stripe_row_limit => 1000);
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 2350) i;
SELECT row_count, first_row_number FROM columnar.stripe a
WHERE a.storage_id = columnar_test_helpers.columnar_relation_storageid('col_table_1'::regclass)
ORDER BY stripe_num;
VACUUM FULL col_table_1;
-- show that we properly update first_row_number after VACUUM FULL
SELECT row_count, first_row_number FROM columnar.stripe a
WHERE a.storage_id = columnar_test_helpers.columnar_relation_storageid('col_table_1'::regclass)
ORDER BY stripe_num;
TRUNCATE col_table_1;
BEGIN;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 16) i;
INSERT INTO col_table_1 SELECT i FROM generate_series(1, 16) i;
COMMIT;
-- show that we start with first_row_number=1 after TRUNCATE
SELECT row_count, first_row_number FROM columnar.stripe a
WHERE a.storage_id = columnar_test_helpers.columnar_relation_storageid('col_table_1'::regclass)
ORDER BY stripe_num;
SET client_min_messages TO ERROR;
DROP SCHEMA columnar_first_row_number CASCADE;

View File

@ -17,13 +17,283 @@ explain insert into t values (1, 2);
insert into t values (1, 2);
SELECT * FROM t;
-- create index without the concurrent option. We should
-- error out during index creation.
create index t_idx on t(a, b);
\d t
explain insert into t values (1, 2);
insert into t values (3, 4);
SELECT * FROM t;
-- make sure that we test index scan
set columnar.enable_custom_scan to 'off';
set enable_seqscan to off;
CREATE table columnar_table (a INT, b int) USING columnar;
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(0, 16000) i;
-- unique --
BEGIN;
INSERT INTO columnar_table VALUES (100000000);
SAVEPOINT s1;
-- errors out due to unflushed data in upper transaction
CREATE UNIQUE INDEX ON columnar_table (a);
ROLLBACK;
CREATE UNIQUE INDEX ON columnar_table (a);
BEGIN;
INSERT INTO columnar_table VALUES (16050);
SAVEPOINT s1;
-- index scan errors out due to unflushed data in upper transaction
SELECT a FROM columnar_table WHERE a = 16050;
ROLLBACK;
EXPLAIN (COSTS OFF) SELECT * FROM columnar_table WHERE a=6456;
EXPLAIN (COSTS OFF) SELECT a FROM columnar_table WHERE a=6456;
SELECT (SELECT a FROM columnar_table WHERE a=6456 limit 1)=6456;
SELECT (SELECT b FROM columnar_table WHERE a=6456 limit 1)=6456*2;
-- even if a=16050 doesn't exist, we try to insert it twice so this should error out
INSERT INTO columnar_table VALUES (16050), (16050);
-- should work
INSERT INTO columnar_table VALUES (16050);
-- check edge cases around stripe boundaries, error out
INSERT INTO columnar_table VALUES (16050);
INSERT INTO columnar_table VALUES (15999);
DROP INDEX columnar_table_a_idx;
CREATE TABLE partial_unique_idx_test (a INT, b INT) USING columnar;
CREATE UNIQUE INDEX ON partial_unique_idx_test (a)
WHERE b > 500;
-- should work since b =< 500 and our partial index doesn't check this interval
INSERT INTO partial_unique_idx_test VALUES (1, 2), (1, 2);
-- should work since our partial index wouldn't cover the tuples that we inserted above
INSERT INTO partial_unique_idx_test VALUES (1, 800);
INSERT INTO partial_unique_idx_test VALUES (4, 600);
-- should error out due to (4, 600)
INSERT INTO partial_unique_idx_test VALUES (4, 700);
-- btree --
CREATE INDEX ON columnar_table (a);
SELECT (SELECT SUM(b) FROM columnar_table WHERE a>700 and a<965)=439560;
CREATE INDEX ON columnar_table (b)
WHERE (b > 30000 AND b < 33000);
-- partial index should be way smaller than the non-partial index
SELECT pg_total_relation_size('columnar_table_b_idx') * 5 <
pg_total_relation_size('columnar_table_a_idx');
-- can't use index scan due to partial index boundaries
EXPLAIN (COSTS OFF) SELECT b FROM columnar_table WHERE b = 30000;
-- can use index scan
EXPLAIN (COSTS OFF) SELECT b FROM columnar_table WHERE b = 30001;
-- some more rows
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(16000, 17000) i;
DROP INDEX columnar_table_a_idx;
TRUNCATE columnar_table;
-- pkey --
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(16000, 16499) i;
ALTER TABLE columnar_table ADD PRIMARY KEY (a);
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(16500, 17000) i;
BEGIN;
INSERT INTO columnar_table (a) SELECT 1;
ROLLBACK;
-- should work
INSERT INTO columnar_table (a) SELECT 1;
-- error out
INSERT INTO columnar_table VALUES (16100), (16101);
INSERT INTO columnar_table VALUES (16999);
BEGIN;
REINDEX INDEX columnar_table_pkey;
-- should error even after reindex
INSERT INTO columnar_table VALUES (16999);
ROLLBACK;
VACUUM FULL columnar_table;
-- show that we don't support clustering columnar tables using indexes
CLUSTER columnar_table USING columnar_table_pkey;
ALTER TABLE columnar_table CLUSTER ON columnar_table_pkey;
CLUSTER columnar_table;
-- should error even after vacuum
INSERT INTO columnar_table VALUES (16999);
TRUNCATE columnar_table;
INSERT INTO columnar_table (a, b) SELECT i,i*2 FROM generate_series(1, 160000) i;
SELECT (SELECT b FROM columnar_table WHERE a = 150000)=300000;
TRUNCATE columnar_table;
ALTER TABLE columnar_table DROP CONSTRAINT columnar_table_pkey;
-- hash --
INSERT INTO columnar_table (a, b) SELECT i*2,i FROM generate_series(1, 8000) i;
CREATE INDEX hash_idx ON columnar_table USING HASH (b);
BEGIN;
CREATE INDEX hash_idx_fill_factor ON columnar_table USING HASH (b) WITH (fillfactor=10);
-- same hash index with lower fillfactor should be way bigger
SELECT pg_total_relation_size ('hash_idx_fill_factor') >
pg_total_relation_size ('hash_idx') * 5;
ROLLBACK;
BEGIN;
INSERT INTO columnar_table (a, b) SELECT i*3,i FROM generate_series(1, 8000) i;
ROLLBACK;
INSERT INTO columnar_table (a, b) SELECT i*4,i FROM generate_series(1, 8000) i;
SELECT SUM(a)=42000 FROM columnar_table WHERE b = 7000;
BEGIN;
REINDEX TABLE columnar_table;
SELECT SUM(a)=42000 FROM columnar_table WHERE b = 7000;
ROLLBACK;
VACUUM FULL columnar_table;
SELECT SUM(a)=42000 FROM columnar_table WHERE b = 7000;
-- exclusion contraints --
CREATE TABLE exclusion_test (c1 INT,c2 INT, c3 INT, c4 BOX,
EXCLUDE USING btree (c1 WITH =) INCLUDE(c3,c4) WHERE (c1 < 10)) USING columnar;
-- error out since "c1" is "1" for all rows to be inserted
INSERT INTO exclusion_test SELECT 1, 2, 3*x, BOX('4,4,4,4') FROM generate_series(1,3) AS x;
BEGIN;
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(1,3) AS x;
ROLLBACK;
-- should work
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(1,3) AS x;
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(10,15) AS x;
BEGIN;
-- should work thanks to "where" clause in exclusion constraint
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(10,15) AS x;
ROLLBACK;
REINDEX TABLE exclusion_test;
-- should still work after reindex
INSERT INTO exclusion_test SELECT x, 2, 3*x, BOX('4,4,4,4') FROM generate_series(10,15) AS x;
-- make sure that we respect INCLUDE syntax --
CREATE TABLE include_test (a INT, b BIGINT, c BIGINT, d BIGINT) USING columnar;
INSERT INTO include_test SELECT i, i, i, i FROM generate_series (1, 1000) i;
CREATE UNIQUE INDEX unique_a ON include_test (a);
-- cannot use index only scan
EXPLAIN (COSTS OFF) SELECT b FROM include_test WHERE a = 500;
CREATE UNIQUE INDEX unique_a_include_b_c_d ON include_test (a) INCLUDE(b, c, d);
-- same unique index that includes other columns should be way bigger
SELECT pg_total_relation_size ('unique_a') * 1.5 <
pg_total_relation_size ('unique_a_include_b_c_d');
DROP INDEX unique_a;
-- should use index only scan since unique_a_include_b_c_d includes column "b" too
EXPLAIN (COSTS OFF) SELECT b FROM include_test WHERE a = 500;
BEGIN;
SET enable_indexonlyscan = OFF;
-- show that we respect enable_indexonlyscan GUC
EXPLAIN (COSTS OFF) SELECT b FROM include_test WHERE a = 500;
ROLLBACK;
-- make sure that we read the correct value for "b" when doing index only scan
SELECT b=980 FROM include_test WHERE a = 980;
-- some tests with distributed & partitioned tables --
CREATE TABLE dist_part_table(
dist_col INT,
part_col TIMESTAMPTZ,
col1 TEXT
) PARTITION BY RANGE (part_col);
-- create an index before creating a columnar partition
CREATE INDEX dist_part_table_btree ON dist_part_table (col1);
-- columnar partition
CREATE TABLE p0 PARTITION OF dist_part_table
FOR VALUES FROM ('2020-01-01') TO ('2020-02-01')
USING columnar;
SELECT create_distributed_table('dist_part_table', 'dist_col');
-- columnar partition
CREATE TABLE p1 PARTITION OF dist_part_table
FOR VALUES FROM ('2020-02-01') TO ('2020-03-01')
USING columnar;
-- row partition
CREATE TABLE p2 PARTITION OF dist_part_table
FOR VALUES FROM ('2020-03-01') TO ('2020-04-01');
INSERT INTO dist_part_table VALUES (1, '2020-03-15', 'str1', POINT(1, 1));
-- insert into columnar partitions
INSERT INTO dist_part_table VALUES (1, '2020-01-15', 'str2', POINT(2, 2));
INSERT INTO dist_part_table VALUES (1, '2020-02-15', 'str3', POINT(3, 3));
-- create another index after creating a columnar partition
CREATE UNIQUE INDEX dist_part_table_unique ON dist_part_table (dist_col, part_col);
-- verify that indexes are created on columnar partitions
SELECT COUNT(*)=2 FROM pg_indexes WHERE tablename = 'p0';
SELECT COUNT(*)=2 FROM pg_indexes WHERE tablename = 'p1';
-- unsupported index types --
-- gin --
CREATE TABLE testjsonb (j JSONB) USING columnar;
INSERT INTO testjsonb SELECT CAST('{"f1" : ' ||'"'|| i*4 ||'", ' || '"f2" : '||'"'|| i*10 ||'"}' AS JSON) FROM generate_series(1,10) i;
CREATE INDEX jidx ON testjsonb USING GIN (j);
INSERT INTO testjsonb SELECT CAST('{"f1" : ' ||'"'|| i*4 ||'", ' || '"f2" : '||'"'|| i*10 ||'"}' AS JSON) FROM generate_series(15,20) i;
-- gist --
CREATE TABLE gist_point_tbl(id INT4, p POINT) USING columnar;
INSERT INTO gist_point_tbl (id, p) SELECT g, point(g*10, g*10) FROM generate_series(1, 10) g;
CREATE INDEX gist_pointidx ON gist_point_tbl USING gist(p);
INSERT INTO gist_point_tbl (id, p) SELECT g, point(g*10, g*10) FROM generate_series(10, 20) g;
-- sp gist --
CREATE TABLE box_temp (f1 box) USING columnar;
INSERT INTO box_temp SELECT box(point(i, i), point(i * 2, i * 2)) FROM generate_series(1, 10) AS i;
CREATE INDEX box_spgist ON box_temp USING spgist (f1);
INSERT INTO box_temp SELECT box(point(i, i), point(i * 2, i * 2)) FROM generate_series(1, 10) AS i;
-- brin --
CREATE TABLE brin_summarize (value int) USING columnar;
CREATE INDEX brin_summarize_idx ON brin_summarize USING brin (value) WITH (pages_per_range=2);
-- Show that we safely fallback to serial index build.
CREATE TABLE parallel_scan_test(a int) USING columnar WITH ( parallel_workers = 2 );
INSERT INTO parallel_scan_test SELECT i FROM generate_series(1,10) i;
CREATE INDEX ON parallel_scan_test (a);
VACUUM FULL parallel_scan_test;
REINDEX TABLE parallel_scan_test;
SET client_min_messages TO WARNING;
DROP SCHEMA columnar_indexes CASCADE;

View File

@ -22,6 +22,10 @@ select count(*) from test_insert_command_data;
insert into test_insert_command select * from test_insert_command_data;
select count(*) from test_insert_command;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_insert_command');
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
drop table test_insert_command_data;
@ -99,6 +103,10 @@ SELECT
pg_column_size(external), pg_column_size(extended)
FROM test_toast_columnar;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('test_toast_columnar');
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
DROP TABLE test_toast_row;
@ -128,6 +136,10 @@ INSERT INTO zero_col_heap SELECT * FROM zero_col_heap;
INSERT INTO zero_col SELECT * FROM zero_col_heap;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('zero_col');
SELECT relname, stripe_num, chunk_group_count, row_count FROM columnar.stripe a, pg_class b
WHERE columnar_test_helpers.columnar_relation_storageid(b.oid)=a.storage_id AND relname = 'zero_col'
ORDER BY 1,2,3,4;

View File

@ -13,6 +13,10 @@ INSERT INTO t SELECT i, i+1 FROM generate_series(1, 10) i;
ROLLBACK;
SELECT count(*) FROM t;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
-- check stripe metadata also have been rolled-back
SELECT count(*) FROM t_stripes;
@ -28,6 +32,11 @@ INSERT INTO t SELECT i, i+1 FROM generate_series(1, 10) i;
SELECT count(*) FROM t; -- force flush
SAVEPOINT s1;
INSERT INTO t SELECT i, i+1 FROM generate_series(1, 10) i;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
SELECT count(*) FROM t;
ROLLBACK TO SAVEPOINT s1;
SELECT count(*) FROM t;
@ -36,6 +45,10 @@ SELECT count(*) FROM t;
INSERT INTO t SELECT i, i+1 FROM generate_series(1, 10) i;
COMMIT;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
SELECT count(*) FROM t;
SELECT count(*) FROM t_stripes;

View File

@ -5,6 +5,17 @@ CREATE FUNCTION columnar_relation_storageid(relid oid) RETURNS bigint
LANGUAGE C STABLE STRICT
AS 'citus', $$columnar_relation_storageid$$;
CREATE OR REPLACE FUNCTION columnar_storage_info(
rel regclass,
version_major OUT int4,
version_minor OUT int4,
storage_id OUT int8,
reserved_stripe_id OUT int8,
reserved_row_number OUT int8,
reserved_offset OUT int8)
STRICT
LANGUAGE c AS 'citus', $$columnar_storage_info$$;
CREATE FUNCTION compression_type_supported(type text) RETURNS boolean
AS $$
BEGIN

View File

@ -27,8 +27,16 @@ SELECT * FROM columnar_truncate_test;
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('columnar_truncate_test');
TRUNCATE TABLE columnar_truncate_test;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('columnar_truncate_test');
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
SELECT * FROM columnar_truncate_test;

View File

@ -17,6 +17,10 @@ INSERT INTO t SELECT i, i * i FROM generate_series(21, 30) i;
SELECT sum(a), sum(b) FROM t;
SELECT count(*) FROM t_stripes;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
-- vacuum full should merge stripes together
VACUUM FULL t;
@ -25,6 +29,10 @@ SELECT * FROM columnar_test_helpers.chunk_group_consistency;
SELECT sum(a), sum(b) FROM t;
SELECT count(*) FROM t_stripes;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
-- test the case when all data cannot fit into a single stripe
SELECT alter_columnar_table_set('t', stripe_row_limit => 1000);
INSERT INTO t SELECT i, 2 * i FROM generate_series(1,2500) i;
@ -34,6 +42,10 @@ SELECT count(*) FROM t_stripes;
VACUUM FULL t;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
SELECT * FROM columnar_test_helpers.chunk_group_consistency;
SELECT sum(a), sum(b) FROM t;
@ -95,6 +107,9 @@ INSERT INTO t SELECT i / 5 FROM generate_series(1, 1500) i;
COMMIT;
VACUUM VERBOSE t;
select
version_major, version_minor, reserved_stripe_id, reserved_row_number, reserved_offset
from columnar_test_helpers.columnar_storage_info('t');
SELECT * FROM columnar_test_helpers.chunk_group_consistency;

View File

@ -0,0 +1,73 @@
\set upgrade_test_old_citus_version `echo "$upgrade_test_old_citus_version"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int >= 10 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int >= 0
AS upgrade_test_old_citus_version_ge_10_0;
\gset
\if :upgrade_test_old_citus_version_ge_10_0
\else
\q
\endif
-- it's not the best practice to define this here, but we don't want to include
-- columnar_test_helpers in upgrade test schedule
CREATE OR REPLACE FUNCTION columnar_storage_info(
rel regclass,
version_major OUT int4,
version_minor OUT int4,
storage_id OUT int8,
reserved_stripe_id OUT int8,
reserved_row_number OUT int8,
reserved_offset OUT int8)
STRICT
LANGUAGE c AS 'citus', 'columnar_storage_info';
CREATE VIEW columnar_table_stripe_info AS
SELECT columnar_table_storageids.relname relname,
columnar.stripe.stripe_num stripe_num,
columnar.stripe.row_count row_count,
columnar.stripe.first_row_number first_row_number
FROM columnar.stripe,
(
SELECT c.oid relid, c.relname relname, (columnar_storage_info(c.oid)).storage_id relstorageid
FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar'
) columnar_table_storageids
WHERE relstorageid = columnar.stripe.storage_id;
SET search_path TO upgrade_columnar_metapage, public;
-- show that first_row_number values are equal to MAX(row_count) * stripe_num + COLUMNAR_FIRST_ROW_NUMBER
SELECT * FROM columnar_table_stripe_info ORDER BY relname, stripe_num;
-- should work since we upgrade metapages when upgrading schema version
INSERT INTO columnar_table_1 VALUES (3);
-- state of stripe metadata for columnar_table_1 after post-upgrade insert
SELECT * FROM columnar_table_stripe_info WHERE relname = 'columnar_table_1' ORDER BY stripe_num;
-- show that all columnar relation's metapage's are upgraded to "2.0"
SELECT count(*)=0
FROM (SELECT (columnar_storage_info(c.oid)).* t
FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar') t
WHERE t.version_major != 2 and t.version_minor != 0;
-- print metapage for two of the tables
SELECT columnar_storage_info('columnar_table_1');
SELECT columnar_storage_info('columnar_table_2');
-- show that no_data_columnar_table also has metapage after upgrade
SELECT columnar_storage_info('no_data_columnar_table');
-- table is already upgraded, make sure that upgrade_columnar_metapage is no-op
SELECT citus_internal.upgrade_columnar_storage(c.oid)
FROM pg_class c, pg_am a
WHERE c.relam = a.oid AND amname = 'columnar' and relname = 'columnar_table_2';
SELECT columnar_storage_info('columnar_table_2');
VACUUM FULL columnar_table_2;
-- print metapage and stripe metadata after post-upgrade vacuum full
SELECT columnar_storage_info('columnar_table_2');
SELECT * FROM columnar_table_stripe_info WHERE relname = 'columnar_table_2' ORDER BY stripe_num;

View File

@ -0,0 +1,26 @@
\set upgrade_test_old_citus_version `echo "$upgrade_test_old_citus_version"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int >= 10 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int >= 0
AS upgrade_test_old_citus_version_ge_10_0;
\gset
\if :upgrade_test_old_citus_version_ge_10_0
\else
\q
\endif
CREATE SCHEMA upgrade_columnar_metapage;
SET search_path TO upgrade_columnar_metapage, public;
CREATE TABLE columnar_table_1(a INT, b INT) USING columnar;
INSERT INTO columnar_table_1 SELECT i FROM generate_series(160001, 320000) i;
CREATE TABLE columnar_table_2(b INT) USING columnar;
SELECT alter_columnar_table_set('columnar_table_2',
chunk_group_row_limit => 100,
stripe_row_limit => 1000);
INSERT INTO columnar_table_2 SELECT i FROM generate_series(1600, 3500) i;
CREATE TABLE columnar_table_3(b INT) USING columnar;
INSERT INTO columnar_table_3 VALUES (1), (2);
CREATE TABLE no_data_columnar_table(a INT, b INT, c TEXT) USING columnar;