1
0
mirror of https://github.com/mariadb-corporation/mariadb-columnstore-engine.git synced 2025-10-31 18:30:33 +03:00

Merge branch 'stable-23.10' into feat/MCOL-6072-parallel-scan-4-CES-4

This commit is contained in:
Leonid Fedorov
2025-09-09 20:36:06 +04:00
committed by GitHub
17 changed files with 346 additions and 24 deletions

View File

@@ -92,6 +92,8 @@ IFLAG = os.path.join(MCS_ETC_PATH, 'container-initialized')
LIBJEMALLOC_DEFAULT_PATH = os.path.join(MCS_DATA_PATH, 'libjemalloc.so.2') LIBJEMALLOC_DEFAULT_PATH = os.path.join(MCS_DATA_PATH, 'libjemalloc.so.2')
MCS_LOG_PATH = '/var/log/mariadb/columnstore' MCS_LOG_PATH = '/var/log/mariadb/columnstore'
# BRM shmem lock inspection/reset tool
SHMEM_LOCKS_PATH = os.path.join(MCS_INSTALL_BIN, 'mcs-shmem-locks')
# client constants # client constants
CMAPI_PORT = 8640 #TODO: use it in all places CMAPI_PORT = 8640 #TODO: use it in all places

View File

@@ -11,8 +11,9 @@ from time import sleep
import psutil import psutil
from cmapi_server.constants import ( from cmapi_server.constants import (
IFLAG, LIBJEMALLOC_DEFAULT_PATH, MCS_INSTALL_BIN, ALL_MCS_PROGS IFLAG, LIBJEMALLOC_DEFAULT_PATH, MCS_INSTALL_BIN, ALL_MCS_PROGS,
) )
from cmapi_server.process_dispatchers.locks import release_shmem_locks
from cmapi_server.exceptions import CMAPIBasicError from cmapi_server.exceptions import CMAPIBasicError
from cmapi_server.process_dispatchers.base import BaseDispatcher from cmapi_server.process_dispatchers.base import BaseDispatcher
@@ -219,6 +220,11 @@ class ContainerDispatcher(BaseDispatcher):
service_proc = cls._get_proc_object(service) service_proc = cls._get_proc_object(service)
if service == 'workernode': if service == 'workernode':
# Run pre-stop lock reset before saving BRM
# These stale locks can occur if the controllernode couldn't stop correctly
# and they cause mcs-savebrm.py to hang
release_shmem_locks(logger)
# start mcs-savebrm.py before stoping workernode # start mcs-savebrm.py before stoping workernode
logger.debug('Waiting to save BRM.') logger.debug('Waiting to save BRM.')
savebrm_path = os.path.join(MCS_INSTALL_BIN, 'mcs-savebrm.py') savebrm_path = os.path.join(MCS_INSTALL_BIN, 'mcs-savebrm.py')
@@ -289,6 +295,7 @@ class ContainerDispatcher(BaseDispatcher):
...TODO: for next releases. Additional error handling. ...TODO: for next releases. Additional error handling.
""" """
stop_success = True
if cls.is_service_running(service): if cls.is_service_running(service):
# TODO: retry? # TODO: retry?
stop_success = cls.stop(service, is_primary, use_sudo) stop_success = cls.stop(service, is_primary, use_sudo)

View File

@@ -0,0 +1,151 @@
import logging
import re
from dataclasses import dataclass
from time import sleep
from typing import Optional, List
from cmapi_server.constants import SHMEM_LOCKS_PATH
from cmapi_server.process_dispatchers.base import BaseDispatcher
@dataclass
class LocksState:
id: int
name: Optional[str]
readers: int = 0
writers: int = 0
readers_waiting: int = 0
writers_waiting: int = 0
mutex_locked: bool = False
def __str__(self):
name = f"({self.name})" if self.name else ""
return (f"LS {self.id}{name}: {self.readers}r/{self.writers}w" +
f" {self.readers_waiting}rw/{self.writers_waiting}ww m={int(self.mutex_locked)}")
def parse_locks_detail(cmd_output: str, logger: logging.Logger) -> List[LocksState]:
"""Parse detailed per-lock state from mcs-shmem-locks output.
Missing or malformed numeric fields are treated as 0. A logger must be provided
and will be used to emit warnings for malformed lines.
"""
states: List[LocksState] = []
current: Optional[LocksState] = None
current_id = 0
for raw in cmd_output.splitlines():
line = raw.strip()
if not line:
continue
if line.endswith('RWLock'):
# push previous
if current is not None:
states.append(current)
current_id += 1
name = line[:-len('RWLock')].strip() or None
current = LocksState(id=current_id, name=name)
continue
if current is None:
continue
field_specs = [
(r'^readers\s*=\s*(\d+)$', 'readers'),
(r'^writers\s*=\s*(\d+)$', 'writers'),
(r'^readers waiting\s*=\s*(\d+)$', 'readers_waiting'),
(r'^writers waiting\s*=\s*(\d+)$', 'writers_waiting'),
]
matched = False
for pattern, attr in field_specs:
m = re.search(pattern, line)
if m:
try:
setattr(current, attr, int(m.group(1)))
except ValueError:
logger.warning('Failed to parse %s from line: %s', attr, raw)
setattr(current, attr, 0)
matched = True
break
if matched:
continue
m = re.search(r'^mutex locked\s*=\s*(\d+)$', line)
if m:
try:
current.mutex_locked = int(m.group(1)) != 0
except ValueError:
current.mutex_locked = False
continue
# push the last one
if current is not None:
states.append(current)
return states
def release_shmem_locks(logger: logging.Logger, max_iterations: int = 5) -> bool:
"""Attempt to release active shmem locks.
- Inspect all locks.
- Unlock writer lock (there can be only one active, but there can be multiple waiting)
- Unlock each reader lock sequentially
- Re-check and repeat up to max_iterations.
Returns True on success (no active readers/writers remain), False otherwise.
"""
# We adapt attempts/sleep when there are waiting locks to allow promotions
attempt = 0
while True:
attempt += 1
success, out = BaseDispatcher.exec_command(f'{SHMEM_LOCKS_PATH} --lock-id 0')
if not success or not out:
logger.error('Failed to inspect shmem locks during unlock (attempt %d)', attempt)
return False
states = parse_locks_detail(out, logger=logger)
active_total = sum(s.readers + s.writers for s in states)
waiting_total = sum(s.readers_waiting + s.writers_waiting for s in states)
if active_total == 0 and waiting_total == 0:
return True
logger.debug(
'Lock release attempt %d: active=%d waiting=%d; detailed=%s',
attempt, active_total, waiting_total, states
)
for st in states:
if st.writers > 0:
cmd = f'{SHMEM_LOCKS_PATH} -i {st.id} -w -u'
ok, _ = BaseDispatcher.exec_command(cmd)
if not ok:
logger.warning('Failed to unlock writer for lock-id=%d', st.id)
if st.readers > 0:
for _ in range(st.readers):
cmd = f'{SHMEM_LOCKS_PATH} -i {st.id} -r -u'
ok, _ = BaseDispatcher.exec_command(cmd)
if not ok:
logger.warning('Failed to unlock a reader for lock-id=%d', st.id)
break
# Wait for state to settle; longer if we have waiting locks
sleep(2 if waiting_total > 0 else 1)
# Allow more attempts when there are waiting locks
effective_max = max_iterations if waiting_total == 0 else max(max_iterations, 15)
if attempt >= effective_max:
break
logger.error('Failed to fully release shmem locks using mcs-shmem-locks after %d attempts (active/waiting remain)', attempt)
return False
def sum_active_from_states(states: List[LocksState]) -> int:
return sum(s.readers + s.writers for s in states)
def sum_waiting_from_states(states: List[LocksState]) -> int:
return sum(s.readers_waiting + s.writers_waiting for s in states)

View File

@@ -5,6 +5,7 @@ import re
from typing import Union, Tuple from typing import Union, Tuple
from cmapi_server.process_dispatchers.base import BaseDispatcher from cmapi_server.process_dispatchers.base import BaseDispatcher
from cmapi_server.process_dispatchers.locks import release_shmem_locks
class SystemdDispatcher(BaseDispatcher): class SystemdDispatcher(BaseDispatcher):
@@ -164,6 +165,11 @@ class SystemdDispatcher(BaseDispatcher):
""" """
service_name = service service_name = service
if service_name == 'mcs-workernode': if service_name == 'mcs-workernode':
# Run pre-stop lock reset before saving BRM
# These stale locks can occur if the controllernode couldn't stop correctly
# and they cause mcs-savebrm.py to hang
release_shmem_locks(logging.getLogger(__name__))
service_name = f'{service_name}@1.service {service_name}@2.service' service_name = f'{service_name}@1.service {service_name}@2.service'
cls._workernode_enable(False, use_sudo) cls._workernode_enable(False, use_sudo)

View File

@@ -0,0 +1,101 @@
import unittest
import logging
from cmapi_server.process_dispatchers.locks import (
parse_locks_detail,
sum_active_from_states,
sum_waiting_from_states,
)
logger = logging.getLogger(__name__)
class TestShmemLocksParsing(unittest.TestCase):
def test_parse_locks_detail_basic(self):
sample_output = """
VSS RWLock
readers = 2
writers = 1
readers waiting = 0
writers waiting = 0
mutex locked = 0
ExtentMap RWLock
readers = 0
writers = 0
readers waiting = 1
writers waiting = 2
mutex locked = 1
"""
states = parse_locks_detail(sample_output, logger)
self.assertEqual(len(states), 2)
# Check names
self.assertEqual(states[0].name, 'VSS')
self.assertEqual(states[1].name, 'ExtentMap')
self.assertEqual(states[0].id, 1)
self.assertEqual(states[0].readers, 2)
self.assertEqual(states[0].writers, 1)
self.assertEqual(states[0].readers_waiting, 0)
self.assertEqual(states[0].writers_waiting, 0)
self.assertFalse(states[0].mutex_locked)
self.assertEqual(states[1].id, 2)
self.assertEqual(states[1].readers, 0)
self.assertEqual(states[1].writers, 0)
self.assertEqual(states[1].readers_waiting, 1)
self.assertEqual(states[1].writers_waiting, 2)
self.assertTrue(states[1].mutex_locked)
self.assertEqual(sum_active_from_states(states), 3)
self.assertEqual(sum_waiting_from_states(states), 3)
def test_parse_locks_detail_malformed_values(self):
sample_output = """
VSS RWLock
readers = blorg
writers = 1
readers waiting = nope
writers waiting = 0
mutex locked = 0
ExtentMap RWLock
readers = 3
writers = one
readers waiting = 0
writers waiting = two
mutex locked = 1
FreeList RWLock
readers = 1
writers = 0
readers waiting = 0
writers waiting = 0
mutex locked = 0
"""
states = parse_locks_detail(sample_output, logger)
# Malformed numeric fields are treated as 0, sections are retained
self.assertEqual(len(states), 3)
self.assertEqual([s.name for s in states], ['VSS', 'ExtentMap', 'FreeList'])
# Lock 1: readers -> 0, writers -> 1, readers_waiting -> 0
self.assertEqual((states[0].readers, states[0].writers, states[0].readers_waiting), (0, 1, 0))
# Lock 2: writers -> 0, writers_waiting -> 0 due to malformed
self.assertEqual((states[1].readers, states[1].writers, states[1].writers_waiting), (3, 0, 0))
# Lock 3 intact
self.assertEqual((states[2].readers, states[2].writers), (1, 0))
def test_parse_locks_detail_mutex_absent_defaults_false(self):
sample_output = """
VSS RWLock
readers = 1
writers = 0
ExtentMap RWLock
readers = 0
writers = 0
"""
states = parse_locks_detail(sample_output, logger)
self.assertEqual(len(states), 2)
self.assertFalse(states[0].mutex_locked)
self.assertFalse(states[1].mutex_locked)
if __name__ == "__main__":
unittest.main()

View File

@@ -15,11 +15,11 @@ $ mcs [OPTIONS] COMMAND [ARGS]...
**Commands**: **Commands**:
* `backup`: Backup Columnstore and/or MariDB data. * `backup`: Backup Columnstore and/or MariaDB server data.
* `dbrm_backup`: Columnstore DBRM Backup. * `dbrm_backup`: Columnstore DBRM Backup.
* `restore`: Restore Columnstore (and/or MariaDB) data. * `restore`: Restore Columnstore (and/or MariaDB server) data.
* `dbrm_restore`: Restore Columnstore DBRM data. * `dbrm_restore`: Restore Columnstore DBRM data.
* `cskeys`: Generates a random AES encryption key and init vector and writes them to disk. * `cskeys`: Generate a random AES encryption key and init vector and write them to disk.
* `cspasswd`: Encrypt a Columnstore plaintext password. * `cspasswd`: Encrypt a Columnstore plaintext password.
* `bootstrap-single-node`: Bootstrap a single node (localhost)... * `bootstrap-single-node`: Bootstrap a single node (localhost)...
* `review`: Provides useful functions to review and troubleshoot the MCS cluster. * `review`: Provides useful functions to review and troubleshoot the MCS cluster.
@@ -31,11 +31,11 @@ $ mcs [OPTIONS] COMMAND [ARGS]...
* `node`: Cluster nodes management. * `node`: Cluster nodes management.
* `set`: Set cluster parameters. * `set`: Set cluster parameters.
* `cluster`: MariaDB Columnstore cluster management... * `cluster`: MariaDB Columnstore cluster management...
* `cmapi`: CMAPI itself related commands. * `cmapi`: Commands related to CMAPI itself.
## `mcs backup` ## `mcs backup`
Backup Columnstore and/or MariDB data. Backup Columnstore and/or MariaDB data.
**Usage**: **Usage**:
@@ -654,7 +654,7 @@ $ mcs cluster set log-level [OPTIONS]
## `mcs cmapi` ## `mcs cmapi`
CMAPI itself related commands. Commands related to CMAPI itself.
**Usage**: **Usage**:

View File

@@ -39,7 +39,7 @@ app.command(
app.command( app.command(
'cskeys', rich_help_panel='Tools commands', 'cskeys', rich_help_panel='Tools commands',
short_help=( short_help=(
'Generates a random AES encryption key and init vector and writes ' 'Generate a random AES encryption key and init vector and write '
'them to disk.' 'them to disk.'
) )
)(tools_commands.cskeys) )(tools_commands.cskeys)

View File

@@ -284,7 +284,7 @@ def backup(
) )
] = None, ] = None,
): ):
"""Backup Columnstore and/or MariDB data.""" """Backup Columnstore and/or MariaDB data."""
# Local Storage Examples: # Local Storage Examples:
# ./$0 backup -bl /tmp/backups/ -bd Local -s LocalStorage # ./$0 backup -bl /tmp/backups/ -bd Local -s LocalStorage

View File

@@ -14,7 +14,7 @@ from mcs_cluster_tool.decorators import handle_output
logger = logging.getLogger('mcs_cli') logger = logging.getLogger('mcs_cli')
app = typer.Typer( app = typer.Typer(
help='CMAPI itself related commands.' help='Commands related to CMAPI itself.'
) )

View File

@@ -444,6 +444,12 @@ void DiskJoinStep::joinFcn(const uint32_t threadID)
smallNullMem[0].reset(new uint8_t[smallNullRow.getSize()]); smallNullMem[0].reset(new uint8_t[smallNullRow.getSize()]);
smallNullRow.setData(rowgroup::Row::Pointer(smallNullMem[0].get())); smallNullRow.setData(rowgroup::Row::Pointer(smallNullMem[0].get()));
smallNullRow.initToNull(); smallNullRow.initToNull();
funcexp::FuncExpWrapper localFE2;
if (thjs->fe2)
{
// Make thread's own copy to prevent a possible race condition when evaluating expressions
localFE2 = *thjs->fe2;
}
try try
{ {
@@ -462,8 +468,8 @@ void DiskJoinStep::joinFcn(const uint32_t threadID)
{ {
l_largeRG.setData(largeData.get()); l_largeRG.setData(largeData.get());
thjs->joinOneRG(0, joinResults, l_largeRG, l_outputRG, l_largeRow, l_joinFERow, l_outputRow, baseRow, thjs->joinOneRG(0, joinResults, l_largeRG, l_outputRG, l_largeRow, l_joinFERow, l_outputRow, baseRow,
joinMatches, smallRowTemplates, outputDL.get(), &joiners, &colMappings, &fergMappings, joinMatches, smallRowTemplates, outputDL.get(), thjs->fe2 ? &localFE2 : nullptr,
&smallNullMem); &joiners, &colMappings, &fergMappings, &smallNullMem);
if (joinResults.size()) if (joinResults.size())
outputResult(joinResults); outputResult(joinResults);

View File

@@ -1576,7 +1576,7 @@ void TupleHashJoinStep::joinRunnerFcn(uint32_t threadID)
joinerRunnerInputRecordsStats[threadID] += local_inputRG.getRowCount(); joinerRunnerInputRecordsStats[threadID] += local_inputRG.getRowCount();
joinOneRG(threadID, joinedRowData, local_inputRG, local_outputRG, largeRow, joinFERow, joinedRow, joinOneRG(threadID, joinedRowData, local_inputRG, local_outputRG, largeRow, joinFERow, joinedRow,
baseRow, joinMatches, smallRowTemplates, outputDL); baseRow, joinMatches, smallRowTemplates, outputDL, fe2 ? &local_fe : nullptr);
} }
if (fe2) if (fe2)
@@ -1718,7 +1718,8 @@ void TupleHashJoinStep::grabSomeWork(vector<RGData>* work)
void TupleHashJoinStep::joinOneRG( void TupleHashJoinStep::joinOneRG(
uint32_t threadID, vector<RGData>& out, RowGroup& inputRG, RowGroup& joinOutput, Row& largeSideRow, uint32_t threadID, vector<RGData>& out, RowGroup& inputRG, RowGroup& joinOutput, Row& largeSideRow,
Row& joinFERow, Row& joinedRow, Row& baseRow, vector<vector<Row::Pointer> >& joinMatches, Row& joinFERow, Row& joinedRow, Row& baseRow, vector<vector<Row::Pointer> >& joinMatches,
std::shared_ptr<Row[]>& smallRowTemplates, RowGroupDL* outputDL, std::shared_ptr<Row[]>& smallRowTemplates, RowGroupDL* lOutputDL,
FuncExpWrapper* localFE2,
// disk-join support vars. This param list is insane; refactor attempt would be nice at some point. // disk-join support vars. This param list is insane; refactor attempt would be nice at some point.
vector<std::shared_ptr<joiner::TupleJoiner> >* tjoiners, vector<std::shared_ptr<joiner::TupleJoiner> >* tjoiners,
std::shared_ptr<std::shared_ptr<int[]>[]>* rgMappings, std::shared_ptr<std::shared_ptr<int[]>[]>* rgMappings,
@@ -1836,7 +1837,7 @@ void TupleHashJoinStep::joinOneRG(
applyMapping((*rgMappings)[smallSideCount], largeSideRow, &baseRow); applyMapping((*rgMappings)[smallSideCount], largeSideRow, &baseRow);
baseRow.setRid(largeSideRow.getRelRid()); baseRow.setRid(largeSideRow.getRelRid());
generateJoinResultSet(threadID, joinMatches, baseRow, *rgMappings, 0, joinOutput, joinedData, out, generateJoinResultSet(threadID, joinMatches, baseRow, *rgMappings, 0, joinOutput, joinedData, out,
smallRowTemplates, joinedRow, outputDL); smallRowTemplates, joinedRow, lOutputDL, localFE2);
} }
} }
@@ -1850,7 +1851,7 @@ void TupleHashJoinStep::generateJoinResultSet(const uint32_t threadID,
const uint32_t depth, RowGroup& l_outputRG, RGData& rgData, const uint32_t depth, RowGroup& l_outputRG, RGData& rgData,
vector<RGData>& outputData, vector<RGData>& outputData,
const std::shared_ptr<Row[]>& smallRows, Row& joinedRow, const std::shared_ptr<Row[]>& smallRows, Row& joinedRow,
RowGroupDL* dlp) RowGroupDL* dlp, FuncExpWrapper* localFE2)
{ {
uint32_t i; uint32_t i;
Row& smallRow = smallRows[depth]; Row& smallRow = smallRows[depth];
@@ -1863,7 +1864,7 @@ void TupleHashJoinStep::generateJoinResultSet(const uint32_t threadID,
smallRow.setPointer(joinerOutput[depth][i]); smallRow.setPointer(joinerOutput[depth][i]);
applyMapping(mappings[depth], smallRow, &baseRow); applyMapping(mappings[depth], smallRow, &baseRow);
generateJoinResultSet(threadID, joinerOutput, baseRow, mappings, depth + 1, l_outputRG, rgData, generateJoinResultSet(threadID, joinerOutput, baseRow, mappings, depth + 1, l_outputRG, rgData,
outputData, smallRows, joinedRow, dlp); outputData, smallRows, joinedRow, dlp, localFE2);
} }
} }
else else
@@ -1887,7 +1888,7 @@ void TupleHashJoinStep::generateJoinResultSet(const uint32_t threadID,
if (UNLIKELY(outputData.size() > flushThreshold || !getMemory(l_outputRG.getSizeWithStrings()))) if (UNLIKELY(outputData.size() > flushThreshold || !getMemory(l_outputRG.getSizeWithStrings())))
{ {
// MCOL-5512 // MCOL-5512
if (fe2) if (localFE2)
{ {
RowGroup l_fe2RG; RowGroup l_fe2RG;
Row fe2InRow; Row fe2InRow;
@@ -1900,7 +1901,7 @@ void TupleHashJoinStep::generateJoinResultSet(const uint32_t threadID,
// WIP do we remove previosuly pushed(line 1825) rgData // WIP do we remove previosuly pushed(line 1825) rgData
// replacing it with a new FE2 rgdata added by processFE2? // replacing it with a new FE2 rgdata added by processFE2?
// Generates a new RGData w/o accounting its memory consumption // Generates a new RGData w/o accounting its memory consumption
processFE2(l_outputRG, l_fe2RG, fe2InRow, fe2OutRow, &outputData, fe2.get()); processFE2(l_outputRG, l_fe2RG, fe2InRow, fe2OutRow, &outputData, localFE2);
} }
// Don't let the join results buffer get out of control. // Don't let the join results buffer get out of control.
sendResult(outputData); sendResult(outputData);

View File

@@ -545,7 +545,7 @@ class TupleHashJoinStep : public JobStep, public TupleDeliveryStep
rowgroup::RowGroup& outputRG, rowgroup::RGData& rgData, rowgroup::RowGroup& outputRG, rowgroup::RGData& rgData,
std::vector<rowgroup::RGData>& outputData, std::vector<rowgroup::RGData>& outputData,
const std::shared_ptr<rowgroup::Row[]>& smallRows, rowgroup::Row& joinedRow, const std::shared_ptr<rowgroup::Row[]>& smallRows, rowgroup::Row& joinedRow,
RowGroupDL* outputDL); RowGroupDL* outputDL, funcexp::FuncExpWrapper* localFE2);
void grabSomeWork(std::vector<rowgroup::RGData>* work); void grabSomeWork(std::vector<rowgroup::RGData>* work);
void sendResult(const std::vector<rowgroup::RGData>& res); void sendResult(const std::vector<rowgroup::RGData>& res);
void processFE2(rowgroup::RowGroup& input, rowgroup::RowGroup& output, rowgroup::Row& inRow, void processFE2(rowgroup::RowGroup& input, rowgroup::RowGroup& output, rowgroup::Row& inRow,
@@ -555,7 +555,8 @@ class TupleHashJoinStep : public JobStep, public TupleDeliveryStep
rowgroup::RowGroup& joinOutput, rowgroup::Row& largeSideRow, rowgroup::Row& joinFERow, rowgroup::RowGroup& joinOutput, rowgroup::Row& largeSideRow, rowgroup::Row& joinFERow,
rowgroup::Row& joinedRow, rowgroup::Row& baseRow, rowgroup::Row& joinedRow, rowgroup::Row& baseRow,
std::vector<std::vector<rowgroup::Row::Pointer>>& joinMatches, std::vector<std::vector<rowgroup::Row::Pointer>>& joinMatches,
std::shared_ptr<rowgroup::Row[]>& smallRowTemplates, RowGroupDL* outputDL, std::shared_ptr<rowgroup::Row[]>& smallRowTemplates, RowGroupDL* lOutputDL,
funcexp::FuncExpWrapper* localFE2,
std::vector<std::shared_ptr<joiner::TupleJoiner>>* joiners = nullptr, std::vector<std::shared_ptr<joiner::TupleJoiner>>* joiners = nullptr,
std::shared_ptr<std::shared_ptr<int[]>[]>* rgMappings = nullptr, std::shared_ptr<std::shared_ptr<int[]>[]>* rgMappings = nullptr,
std::shared_ptr<std::shared_ptr<int[]>[]>* feMappings = nullptr, std::shared_ptr<std::shared_ptr<int[]>[]>* feMappings = nullptr,

View File

@@ -0,0 +1 @@
--plugin-load-add=$HA_COLUMNSTORE_SO

View File

@@ -0,0 +1,29 @@
package My::Suite::ColumnStore;
@ISA = qw(My::Suite);
my $mcs_bin_dir_compiled=$::bindir . '/storage/columnstore/columnstore/bin';
my $mcs_ins_dir_installed=$::bindir . '/bin';
if (-d $mcs_bin_dir_compiled)
{
$ENV{MCS_MCSSETCONFIG}=$mcs_bin_dir_compiled . "/mcsSetConfig";
$ENV{MCS_CPIMPORT}=$mcs_bin_dir_compiled . "/cpimport";
$ENV{MCS_SYSCATALOG_MYSQL_SQL}=$::mysqld_variables{'basedir'} . "/storage/columnstore/columnstore/dbcon/mysql/syscatalog_mysql.sql";
}
elsif (-d $mcs_ins_dir_installed)
{
$ENV{MCS_MCSSETCONFIG}=$mcs_ins_dir_installed . "/mcsSetConfig";
$ENV{MCS_CPIMPORT}=$mcs_ins_dir_installed . "/cpimport";
$ENV{MCS_SYSCATALOG_MYSQL_SQL}=$::mysqld_variables{'basedir'} . "/share/columnstore/syscatalog_mysql.sql";
}
sub is_default { 0 }
sub start_test {
# we should guard this for --force-restart flag condition.
my ($self, $tinfo)= @_;
My::Suite::start_test(@_);
}
bless { };

View File

@@ -5,6 +5,9 @@
# -------------------------------------------------------------- # # -------------------------------------------------------------- #
# #
--source ../include/have_columnstore.inc --source ../include/have_columnstore.inc
--source ../include/cross_engine.inc
# #
USE tpch1; USE tpch1;
# #
@@ -32,3 +35,6 @@ DROP TABLE test.mcol830b;
--enable_warnings --enable_warnings
# #
--source ../include/drop_cross_engine.inc

View File

@@ -2,8 +2,16 @@
# Enable cross engine join # Enable cross engine join
# Configure user and password in Columnstore.xml file # Configure user and password in Columnstore.xml file
# -------------------------------------------------------------- # # -------------------------------------------------------------- #
--exec /usr/bin/mcsSetConfig CrossEngineSupport User 'cejuser'
--exec /usr/bin/mcsSetConfig CrossEngineSupport Password 'Vagrant1|0000001' --disable_query_log
if (!$MASTER_MYPORT)
{
# Running with --extern
let $MASTER_MYPORT=`SELECT @@port`;
}
--exec $MCS_MCSSETCONFIG CrossEngineSupport User 'cejuser'
--exec $MCS_MCSSETCONFIG CrossEngineSupport Password 'Vagrant1|0000001'
--exec $MCS_MCSSETCONFIG CrossEngineSupport Port $MASTER_MYPORT --exec $MCS_MCSSETCONFIG CrossEngineSupport Port $MASTER_MYPORT
# -------------------------------------------------------------- # # -------------------------------------------------------------- #
@@ -16,4 +24,4 @@ CREATE USER IF NOT EXISTS'cejuser'@'localhost' IDENTIFIED BY 'Vagrant1|0000001';
GRANT ALL PRIVILEGES ON *.* TO 'cejuser'@'localhost'; GRANT ALL PRIVILEGES ON *.* TO 'cejuser'@'localhost';
FLUSH PRIVILEGES; FLUSH PRIVILEGES;
--enable_query_log

View File

@@ -1,2 +1,5 @@
--disable_query_log
REVOKE ALL PRIVILEGES ON *.* FROM 'cejuser'@'localhost'; REVOKE ALL PRIVILEGES ON *.* FROM 'cejuser'@'localhost';
DROP USER 'cejuser'@'localhost'; DROP USER 'cejuser'@'localhost';
--enable_query_log