1
0
mirror of https://github.com/mariadb-corporation/mariadb-columnstore-engine.git synced 2025-07-30 19:23:07 +03:00
* MCOL-4560 remove unused xml entries and code that references it.
There is reader code and variables for some of these settings, but nobody uses them.
This commit is contained in:
David.Hall
2022-04-18 17:00:17 -05:00
committed by GitHub
parent bd4c911ddb
commit bbb168a846
28 changed files with 93 additions and 3516 deletions

View File

@ -95,7 +95,7 @@ void fix_column_length(SchemaObject* elem, const CHARSET_INFO* def_cs) {
%} %}
%expect 17 %expect 17
%pure-parser %define api.pure
%lex-param {void * scanner} %lex-param {void * scanner}
%parse-param {struct ddlpackage::pass_to_bison * x} %parse-param {struct ddlpackage::pass_to_bison * x}

View File

@ -90,7 +90,7 @@ char* copy_string(const char *str);
} }
%} %}
%pure-parser %define api.pure
%lex-param {void * scanner} %lex-param {void * scanner}
%parse-param {void * scanner} %parse-param {void * scanner}
%debug %debug

View File

@ -36,7 +36,6 @@ set(joblist_LIB_SRCS
pcolstep.cpp pcolstep.cpp
pdictionary.cpp pdictionary.cpp
pdictionaryscan.cpp pdictionaryscan.cpp
primitivemsg.cpp
pseudocc-jl.cpp pseudocc-jl.cpp
resourcedistributor.cpp resourcedistributor.cpp
resourcemanager.cpp resourcemanager.cpp

View File

@ -28,7 +28,6 @@
// //
// //
/** @file */ /** @file */
// #include "primitivemsg.h"
#include "bytestream.h" #include "bytestream.h"
#include "messagequeue.h" #include "messagequeue.h"
#include "serializeable.h" #include "serializeable.h"

View File

@ -199,7 +199,7 @@ void DistributedEngineComm::reset()
} }
DistributedEngineComm::DistributedEngineComm(ResourceManager* rm, bool isExeMgr) DistributedEngineComm::DistributedEngineComm(ResourceManager* rm, bool isExeMgr)
: fRm(rm), fLBIDShift(fRm->getPsLBID_Shift()), pmCount(0), fIsExeMgr(isExeMgr) : fRm(rm), pmCount(0), fIsExeMgr(isExeMgr)
{ {
Setup(); Setup();
} }
@ -250,10 +250,6 @@ void DistributedEngineComm::Setup()
if (newPmCount == 0) if (newPmCount == 0)
writeToLog(__FILE__, __LINE__, "Got a config file with 0 PMs", LOG_TYPE_CRITICAL); writeToLog(__FILE__, __LINE__, "Got a config file with 0 PMs", LOG_TYPE_CRITICAL);
// This needs to make sense when compared to the extent size
// fLBIDShift = static_cast<unsigned>(config::Config::uFromText(fConfig->getConfig(section,
// "LBID_Shift")));
auto* config = fRm->getConfig(); auto* config = fRm->getConfig();
std::vector<messageqcpp::AddrAndPortPair> pmsAddressesAndPorts; std::vector<messageqcpp::AddrAndPortPair> pmsAddressesAndPorts;
for (size_t i = 1; i <= newPmCount; ++i) for (size_t i = 1; i <= newPmCount; ++i)

View File

@ -269,7 +269,6 @@ class DistributedEngineComm
boost::mutex fMlock; // sessionMessages mutex boost::mutex fMlock; // sessionMessages mutex
std::vector<boost::shared_ptr<boost::mutex> > fWlock; // PrimProc socket write mutexes std::vector<boost::shared_ptr<boost::mutex> > fWlock; // PrimProc socket write mutexes
bool fBusy; bool fBusy;
unsigned fLBIDShift;
volatile uint32_t pmCount; volatile uint32_t pmCount;
boost::mutex fOnErrMutex; // to lock function scope to reset pmconnections under error condition boost::mutex fOnErrMutex; // to lock function scope to reset pmconnections under error condition
boost::mutex fSetupMutex; boost::mutex fSetupMutex;

View File

@ -185,14 +185,8 @@ struct JobInfo
, maxElems(rm->getHjMaxElems()) , maxElems(rm->getHjMaxElems())
, flushInterval(rm->getJLFlushInterval()) , flushInterval(rm->getJLFlushInterval())
, fifoSize(rm->getJlFifoSize()) , fifoSize(rm->getJlFifoSize())
, fifoSizeLargeSideHj(rm->getHjFifoSizeLargeSide())
, scanLbidReqLimit(rm->getJlScanLbidReqLimit())
, scanLbidReqThreshold(rm->getJlScanLbidReqThreshold())
, tempSaveSize(rm->getScTempSaveSize())
, logger(new Logger()) , logger(new Logger())
, traceFlags(0) , traceFlags(0)
, tupleDLMaxSize(rm->getTwMaxSize())
, tupleMaxBuckets(rm->getTwMaxBuckets())
, projectingTableOID(0) , projectingTableOID(0)
, isExeMgr(false) , isExeMgr(false)
, trace(false) , trace(false)
@ -226,18 +220,8 @@ struct JobInfo
JobStepVectorStack stack; JobStepVectorStack stack;
uint32_t flushInterval; uint32_t flushInterval;
uint32_t fifoSize; uint32_t fifoSize;
uint32_t fifoSizeLargeSideHj;
//...joblist does not use scanLbidReqLimit and SdanLbidReqThreshold.
//...They are actually used by pcolscan and pdictionaryscan, but
//...we have joblist get and report the values here since they
//...are global to the job.
uint32_t scanLbidReqLimit;
uint32_t scanLbidReqThreshold;
uint32_t tempSaveSize;
SPJL logger; SPJL logger;
uint32_t traceFlags; uint32_t traceFlags;
uint64_t tupleDLMaxSize;
uint32_t tupleMaxBuckets;
SErrorInfo errorInfo; SErrorInfo errorInfo;
execplan::CalpontSystemCatalog::OID* projectingTableOID; // DeliveryWSDLs get a reference to this execplan::CalpontSystemCatalog::OID* projectingTableOID; // DeliveryWSDLs get a reference to this
bool isExeMgr; bool isExeMgr;

View File

@ -2097,9 +2097,7 @@ SJLP makeJobList_(CalpontExecutionPlan* cplan, ResourceManager* rm, bool isExeMg
oss << endl; oss << endl;
oss << endl << "job parms: " << endl; oss << endl << "job parms: " << endl;
oss << "maxBuckets = " << jobInfo.maxBuckets << ", maxElems = " << jobInfo.maxElems oss << "maxBuckets = " << jobInfo.maxBuckets << ", maxElems = " << jobInfo.maxElems
<< ", flushInterval = " << jobInfo.flushInterval << ", fifoSize = " << jobInfo.fifoSize << ", flushInterval = " << jobInfo.flushInterval << ", fifoSize = " << jobInfo.fifoSize << endl;
<< ", ScanLimit/Threshold = " << jobInfo.scanLbidReqLimit << "/" << jobInfo.scanLbidReqThreshold
<< endl;
oss << "UUID: " << jobInfo.uuid << endl; oss << "UUID: " << jobInfo.uuid << endl;
oss << endl << "job filter steps: " << endl; oss << endl << "job filter steps: " << endl;
ostream_iterator<JobStepVector::value_type> oIter(oss, "\n"); ostream_iterator<JobStepVector::value_type> oIter(oss, "\n");

View File

@ -53,64 +53,6 @@ using namespace execplan;
//#define DEBUG 1 //#define DEBUG 1
//#define DEBUG2 1 //#define DEBUG2 1
namespace
{
//// const uint32_t defaultScanLbidReqLimit = 10000;
//// const uint32_t defaultScanLbidReqThreshold = 5000;
//
// struct pColScanStepPrimitive
//{
// pColScanStepPrimitive(pColScanStep* pColScanStep) : fPColScanStep(pColScanStep)
// {}
// pColScanStep *fPColScanStep;
// void operator()()
// {
// try
// {
// fPColScanStep->sendPrimitiveMessages();
// }
// catch(std::exception& re)
// {
// string msg = re.what();
// cerr << "pColScanStep: send thread threw an exception: " << msg << endl;
//
// //Whoa! is this really what we want to do? It's not clear that any good can be had by
// //sticking around, but this seems drastic...
// if (msg.find("there are no primitive processors") != string::npos)
// {
// SPJL logger = fPColScanStep->logger();
// logger->logMessage(LOG_TYPE_CRITICAL, LogNoPrimProcs, Message::Args(),
//LoggingID(5)); exit(1);
// }
// }
// }
//};
//
// struct pColScanStepAggregater
//{
// pColScanStepAggregater(pColScanStep* pColScanStep, uint64_t index) :
// fPColScanStepCol(pColScanStep), fThreadId(index)
// {}
// pColScanStep *fPColScanStepCol;
// uint64_t fThreadId;
//
// void operator()()
// {
// try
// {
// fPColScanStepCol->receivePrimitiveMessages(fThreadId);
// }
// catch(std::exception& re)
// {
// cerr << fPColScanStepCol->toString() << ": receive thread threw an exception: " <<
//re.what() << endl;
// }
// }
//};
//
} // namespace
namespace joblist namespace joblist
{ {
pColScanStep::pColScanStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OID t, pColScanStep::pColScanStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OID t,
@ -118,20 +60,11 @@ pColScanStep::pColScanStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OI
: JobStep(jobInfo) : JobStep(jobInfo)
, fRm(jobInfo.rm) , fRm(jobInfo.rm)
, fMsgHeader() , fMsgHeader()
, fNumThreads(fRm->getJlNumScanReceiveThreads())
, fFilterCount(0) , fFilterCount(0)
, fOid(o) , fOid(o)
, fTableOid(t) , fTableOid(t)
, fColType(ct) , fColType(ct)
, fBOP(BOP_OR) , fBOP(BOP_OR)
, sentCount(0)
, recvCount(0)
, fScanLbidReqLimit(fRm->getJlScanLbidReqLimit())
, fScanLbidReqThreshold(fRm->getJlScanLbidReqThreshold())
, fStopSending(false)
, fSingleThread(false)
, fPhysicalIO(0)
, fCacheIO(0)
, fNumBlksSkipped(0) , fNumBlksSkipped(0)
, fMsgBytesIn(0) , fMsgBytesIn(0)
, fMsgBytesOut(0) , fMsgBytesOut(0)
@ -234,332 +167,6 @@ pColScanStep::pColScanStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OI
throw runtime_error("pColScan: Block size and column width must be a power of 2"); throw runtime_error("pColScan: Block size and column width must be a power of 2");
} }
pColScanStep::~pColScanStep()
{
// pthread_mutex_destroy(&mutex);
// pthread_mutex_destroy(&dlMutex);
// pthread_mutex_destroy(&cpMutex);
// pthread_cond_destroy(&condvar);
// pthread_cond_destroy(&condvarWakeupProducer);
// delete lbidList;
// delete [] fProducerThread;
// if (fDec)
// fDec->removeQueue(uniqueID); // in case it gets aborted
}
//------------------------------------------------------------------------------
// Initialize configurable parameters
//------------------------------------------------------------------------------
void pColScanStep::initializeConfigParms()
{
// const string section ( "JobList" );
// const string sendLimitName ( "ScanLbidReqLimit" );
// const string sendThresholdName ( "ScanLbidReqThreshold" );
// const string numReadThreadsName ( "NumScanReceiveThreads" );
// Config* cf = Config::makeConfig();
// string strVal;
//...Get the tuning parameters that throttle msgs sent to primproc
//...fScanLbidReqLimit puts a cap on how many LBID's we will request from
//... primproc, before pausing to let the consumer thread catch up.
//... Without this limit, there is a chance that PrimProc could flood
//... ExeMgr with thousands of messages that will consume massive
//... amounts of memory for a 100 gigabyte database.
//...fScanLbidReqThreshold is the level at which the number of outstanding
//... LBID reqs must fall below, before the producer can send more LBIDs.
// strVal = cf->getConfig(section, sendLimitName);
// if (strVal.size() > 0)
// fScanLbidReqLimit = static_cast<uint32_t>(Config::uFromText(strVal));
//
// strVal = cf->getConfig(section, sendThresholdName);
// if (strVal.size() > 0)
// fScanLbidReqThreshold = static_cast<uint32_t>(Config::uFromText(strVal));
//
// fNumThreads = 8;
// strVal = cf->getConfig(section, numReadThreadsName);
// if (strVal.size() > 0)
// fNumThreads = static_cast<uint32_t>(Config::uFromText(strVal));
// fProducerThread = new SPTHD[fNumThreads];
}
void pColScanStep::startPrimitiveThread()
{
// fConsumerThread.reset(new boost::thread(pColScanStepPrimitive(this)));
}
void pColScanStep::startAggregationThread()
{
// for (uint32_t i = 0; i < fNumThreads; i++)
// fProducerThread[i].reset(new boost::thread(pColScanStepAggregater(this, i)));
}
void pColScanStep::run()
{
// if (traceOn())
// {
// syslogStartStep(16, // exemgr subsystem
// std::string("pColScanStep")); // step name
// }
//
// //"consume" input datalist. In this case, there is no IDL, we just send one primitive
// startPrimitiveThread();
// //produce output datalist
// //Don't start this yet...see below
// //startAggregationThread();
}
void pColScanStep::join()
{
// fConsumerThread->join();
// if ( !fSingleThread ) {
// for (uint32_t i = 0; i < fNumThreads; i++)
// fProducerThread[i]->join();
// }
}
void pColScanStep::sendPrimitiveMessages()
{
// //The presence of an input DL means we (probably) have a pDictionaryScan step feeding this scan step
// // a list of tokens to get the rids for. Convert the input tokens to a filter string.
// if (fInputJobStepAssociation.outSize() > 0)
// {
// addFilters();
// if (fTableOid >= 3000)
// cout << toString() << endl;
// //If we got no input rids (as opposed to no input DL at all) then there were no matching rows from
// // the previous step, so this step should not return any rows either. This would be the case, for
// // instance, if P_NAME LIKE '%xxxx%' produced no signature matches.
// if (fFilterCount == 0) {
// rDoNothing=true;
// startAggregationThread();
// return;
// }
// }
//
// startAggregationThread();
//
// /* for all the blocks that need to be sent
// * build out a message for each block with the primitive message header filled
// * out and all the NOPS and BOP structures as well.
// * Then serialize it to a BytStream and then send it on its way
// */
//
// LBIDRange_v::iterator it;
// uint64_t fbo;
//
// ISMPacketHeader ism;
// ism.Flags = planFlagsToPrimFlags(fTraceFlags);
// ism.Command=COL_BY_SCAN_RANGE;
// ism.Size = sizeof(ISMPacketHeader) + sizeof(ColByScanRangeRequestHeader) + fFilterString.length();
// ism.Type=2;
// //bool firstWrite = true;
//
// //...Counter used to track the number of LBIDs we are requesting from
// //...primproc in the current set of msgs, till we reach fScanLbidReqLimit
// uint32_t runningLbidCount = 0;
// bool exitLoop = false;
// const bool ignoreCP = ((fTraceFlags & CalpontSelectExecutionPlan::IGNORE_CP) != 0);
//
// for (it = lbidRanges.begin(); it != lbidRanges.end(); it++)
// {
// BRM::LBID_t lbid = (*it).start;
//
// fbo = getFBO(lbid);
// if (hwm < fbo)
// continue;
//
// if (fOid >= 3000 && lbidList->CasualPartitionDataType(fColType.colDataType, fColType.colWidth) )
// {
// int64_t Min=0;
// int64_t Max=0;
// int64_t SeqNum=0;
// bool MinMaxValid=true;
// bool cpPredicate=true;
//
// // can we consolidate these crit sections?
// cpMutex.lock(); //pthread_mutex_lock(&cpMutex);
// MinMaxValid = lbidList->GetMinMax(Min, Max, SeqNum, lbid, 0);
//
// if (MinMaxValid)
// {
// cpPredicate=lbidList->CasualPartitionPredicate(Min,
// Max,
// &fFilterString,
// fFilterCount,
// fColType,
// fBOP) ||
//ignoreCP;
// }
// cpMutex.unlock(); //pthread_mutex_unlock(&cpMutex);
//
// if (cpPredicate==false){ //don't scan this extent
//#ifdef DEBUG
// cout << "Scan Skip " << lbid << endl;
//#endif
// //...Track the number of LBIDs we skip due to Casual Partioning.
// //...We use the same equation we use to initialize remainingLbids
// //...in the code that follows down below this.
// fNumBlksSkipped += ( (hwm > (fbo + it->size - 1)) ?
// (it->size) : (hwm - fbo + 1) );
// continue;
// }
//#ifdef DEBUG
// else
// cout << "Scan " << lbid << endl;
//#endif
//
// }
//
// LBID_t msgLbidStart = it->start;
// uint32_t remainingLbids =
// ( (hwm > (fbo + it->size - 1)) ? (it->size) : (hwm - fbo + 1) );
// uint32_t msgLbidCount = 0;
//
// while ( remainingLbids > 0 )
// {
// //...Break up this range of LBIDs when we reach the msg size
// //...limit for one request (fScanLbidReqLimit)
// if ( (runningLbidCount + remainingLbids) >= fScanLbidReqLimit )
// {
// msgLbidCount = fScanLbidReqLimit - runningLbidCount;
// sendAPrimitiveMessage(ism, msgLbidStart, msgLbidCount );
// //...Wait for the consuming thread to catch up if our
// //...backlog of work is >= the allowable threshold limit
//
// mutex.lock(); //pthread_mutex_lock(&mutex);
// sentCount += msgLbidCount;
// if (recvWaiting)
// condvar.notify_all(); //pthread_cond_broadcast(&condvar); //signal consumer to
//resume
//
// while ( ((sentCount - recvCount) >= fScanLbidReqThreshold)
// && !fStopSending )
// {
// sendWaiting = true;
//#ifdef DEBUG2
// if (fOid >= 3000)
// cout << "pColScanStep producer WAITING: " <<
// "st:" << fStepId <<
// "; sentCount-" << sentCount <<
// "; recvCount-" << recvCount <<
// "; threshold-" << fScanLbidReqThreshold << endl;
//#endif
// condvarWakeupProducer.wait(mutex); //pthread_cond_wait ( &condvarWakeupProducer, &mutex
//); #ifdef DEBUG2 if (fOid >= 3000) cout << "pColScanStep producer RESUMING: " << "st:" << fStepId << endl;
//#endif
// sendWaiting = false;
// }
//
// //...Set flag to quit if consumer thread tells us to
// if (fStopSending)
// exitLoop = true;
//
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// runningLbidCount = 0;
// }
// else
// {
// msgLbidCount = remainingLbids;
//
// sendAPrimitiveMessage(ism, msgLbidStart, msgLbidCount );
// mutex.lock(); //pthread_mutex_lock(&mutex);
// sentCount += msgLbidCount;
// if (recvWaiting)
// condvar.notify_all(); //pthread_cond_broadcast(&condvar); //signal consumer to
//resume
//
// //...Set flag to quit if consumer thread tells us to
// if (fStopSending)
// exitLoop = true;
//
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// runningLbidCount += msgLbidCount;
// }
//
// //...If consuming thread has quit, then we should do the same.
// //...This can happen if consuming thread receives empty ByteStream
// if (exitLoop)
// break;
//
// remainingLbids -= msgLbidCount;
// msgLbidStart += msgLbidCount;
// }
//
// if (exitLoop)
// break;
//
// } // end of loop through LBID ranges to be requested from primproc
//
// mutex.lock(); //pthread_mutex_lock(&mutex);
// finishedSending = true;
// if (recvWaiting)
// condvar.notify_all(); //pthread_cond_broadcast(&condvar);
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//// cerr << "send side exiting" << endl;
//
//#ifdef DEBUG2
// if (fOid >= 3000)
// {
// time_t t = time(0);
// char timeString[50];
// ctime_r(&t, timeString);
// timeString[strlen(timeString)-1 ] = '\0';
// cout << "pColScanStep Finished sending primitives for: " <<
// fOid << " at " << timeString << endl;
// }
//#endif
//
}
//------------------------------------------------------------------------------
// Construct and send a single primitive message to primproc
//------------------------------------------------------------------------------
void pColScanStep::sendAPrimitiveMessage(ISMPacketHeader& ism, BRM::LBID_t msgLbidStart,
uint32_t msgLbidCount)
{
// ByteStream bs;
//
// bs.load(reinterpret_cast<const ByteStream::byte*>(&ism), sizeof(ism));
//
// fMsgHeader.LBID = msgLbidStart;
// fMsgHeader.DataSize = fColType.colWidth;
// fMsgHeader.DataType = fColType.colDataType;
// fMsgHeader.CompType = fColType.compressionType;
// if (fFilterCount > 0)
// fMsgHeader.OutputType = 3; // <rid, value> pairs
// else
// fMsgHeader.OutputType = OT_DATAVALUE;
// fMsgHeader.BOP = fBOP;
// fMsgHeader.NOPS = fFilterCount;
// fMsgHeader.NVALS = 0;
// fMsgHeader.Count = msgLbidCount;
// fMsgHeader.Hdr.SessionID = fSessionId;
// //fMsgHeader.Hdr.StatementID = 0;
// fMsgHeader.Hdr.TransactionID = fTxnId;
// fMsgHeader.Hdr.VerID = fVerId;
// fMsgHeader.Hdr.StepID = fStepId;
// fMsgHeader.Hdr.UniqueID = uniqueID;
//
// bs.append(reinterpret_cast<const ByteStream::byte*>(&fMsgHeader),
// sizeof(fMsgHeader));
// bs += fFilterString;
//
//#ifdef DEBUG2
// if (fOid >= 3000)
// cout << "pColScanStep producer st: " << fStepId <<
// ": sending req for lbid start " << msgLbidStart <<
// "; lbid count " << msgLbidCount << endl;
//#endif
//
// fMsgBytesOut += bs.lengthWithHdrOverhead();
// fDec->write(bs);
// fMsgsToPm++;
}
struct CPInfo struct CPInfo
{ {
CPInfo(int64_t MIN, int64_t MAX, uint64_t l) : min(MIN), max(MAX), LBID(l){}; CPInfo(int64_t MIN, int64_t MAX, uint64_t l) : min(MIN), max(MAX), LBID(l){};
@ -568,327 +175,6 @@ struct CPInfo
uint64_t LBID; uint64_t LBID;
}; };
void pColScanStep::receivePrimitiveMessages(uint64_t tid)
{
// AnyDataListSPtr dl = fOutputJobStepAssociation.outAt(0);
// DataList_t* dlp = dl->dataList();
// FifoDataList *fifo = dl->fifoDL();
// BucketDL<ElementType> *bucket = dynamic_cast<BucketDL<ElementType> *>(dlp);
// ZDL<ElementType> *zdl = dynamic_cast<ZDL<ElementType> *>(dlp);
// int64_t l_ridsReturned = 0;
// uint64_t l_physicalIO = 0, l_cachedIO = 0;
// uint64_t fbo;
// uint64_t ridBase;
// vector<ElementType> v;
// UintRowGroup rw;
// vector<boost::shared_ptr<ByteStream> > bsv;
// uint32_t i, k, size, bsLength;
// bool lastThread = false;
// vector<CPInfo> cpv;
//
// if (bucket || zdl)
// dlp->setMultipleProducers(true);
//
// mutex.lock(); //pthread_mutex_lock(&mutex);
//
// // count the LBIDs
// for (; !rDoNothing; ) {
//
// // sync with the send side
// while (!finishedSending && sentCount == recvCount) {
// recvWaiting++;
// condvar.wait(mutex); //pthread_cond_wait(&condvar, &mutex);
// recvWaiting--;
// }
// if (sentCount == recvCount && finishedSending) {
//// cout << "done recving" << endl;
// break;
// }
//
// fDec->read_some(uniqueID, fNumThreads, bsv);
// for (unsigned int jj=0; jj<bsv.size(); jj++) {
// fMsgBytesIn += bsv[jj]->lengthWithHdrOverhead();
// }
//
//
// size = bsv.size();
//
// if (size == 0) {
// /* XXXPAT: Need to give other threads a chance to update recvCount.
// As of 2/25/08, each BS contains multiple responses. With the current
// protocol, the exact # isn't known until they're processed. Threads
// waiting for more input will have to busy wait on either recvCount
// being updated or input. It's only an issue at the tail end of the
// responses, and probably doesn't matter then either. */
//
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
// usleep(1000); // 1ms Good?
// mutex.lock(); //pthread_mutex_lock(&mutex);
// continue;
// }
//
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0)
// dlTimes.setFirstReadTime();
// if (fOid>=3000) dlTimes.setLastReadTime();
//
//// cerr << "got a response of " << size << " msgs\n";
//
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// uint32_t msgCount = 0;
// for (i = 0; i < size; i++) {
// const ByteStream::byte* bsp = bsv[i]->buf();
//
// bsLength = bsv[i]->length();
// k = 0;
// while (k < bsLength) {
// ++msgCount;
//// cout << "got msg " << msgCount << " k = " << k << endl;
// k += sizeof(ISMPacketHeader);
// const ColResultHeader* crh = reinterpret_cast<const
//ColResultHeader*>(&bsp[k]);
// // get the ColumnResultHeader out of the bytestream
// k += sizeof(ColResultHeader);
//
// l_cachedIO += crh->CacheIO;
// l_physicalIO += crh->PhysicalIO;
// fbo = getFBO(crh->LBID);
// ridBase = fbo << rpbShift;
//
// for(int j = 0; j < crh->NVALS; j++)
// {
// uint64_t dv;
// uint64_t rid;
//
// if (crh->OutputType == OT_DATAVALUE) {
// if (isEmptyVal(&bsp[k])) {
// k += fColType.colWidth;
// continue;
// }
// rid = j + ridBase;
// }
// else {
// rid = *((const uint16_t *) &bsp[k]) + ridBase;
// k += sizeof(uint16_t);
// }
//
// switch (fColType.colWidth) {
// case 8: dv = *((const uint64_t *) &bsp[k]); k += 8; break;
// case 4: dv = *((const uint32_t *) &bsp[k]); k += 4; break;
// case 2: dv = *((const uint16_t *) &bsp[k]); k += 2; break;
// case 1: dv = *((const uint8_t *) &bsp[k]); ++k; break;
// default:
// throw runtime_error("pColStep: invalid column
//width!");
// }
//
// v.push_back(ElementType(rid, dv));
// ++l_ridsReturned;
//#ifdef DEBUG
//// if (fOid >=3000)
//// cout << " -- inserting <" << rid << ", " << dv << ">" <<
///endl;
//#endif
// // per row operations...
// } // for
//
// // per block operations...
//
//#ifdef DEBUG
// cout << "recvPrimMsgs Oid " << fOid
// << " valid " << crh->ValidMinMax
// << " LBID " << crh->LBID
// << " Mn/Mx " << crh->Min << "/" << crh->Max
// << " nvals " << crh->NVALS
// << "/" << l_ridsReturned << endl;
//#endif
// if (fOid >= 3000 && crh->ValidMinMax)
// cpv.push_back(CPInfo(crh->Min, crh->Max, crh->LBID));
// }
// // per ByteStream operations...
// }
// // per read operations....
//
// if (bucket) {
// if (fOid>=3000 && dlTimes.FirstInsertTime().tv_sec==0)
// dlTimes.setFirstInsertTime();
//
// bucket->insert(v);
// }
// else if (zdl) {
// zdl->insert(v);
// }
// else {
// size = v.size();
// if (size>0)
// if (fOid>=3000 && dlTimes.FirstInsertTime().tv_sec==0)
// dlTimes.setFirstInsertTime();
//
// dlMutex.lock(); //pthread_mutex_lock(&dlMutex);
// for (i = 0; i < size; ++i) {
// rw.et[rw.count++] = v[i];
// if (rw.count == rw.ElementsPerGroup)
// {
// fifo->insert(rw);
// rw.count = 0;
// }
// }
//
// if (rw.count > 0)
// {
// fifo->insert(rw);
// rw.count = 0;
// }
//
// dlMutex.unlock(); //pthread_mutex_unlock(&dlMutex);
// }
// v.clear();
//
// size = cpv.size();
// if (size > 0) {
// cpMutex.lock(); //pthread_mutex_lock(&cpMutex);
// for (i = 0; i < size; i++) {
// CPInfo *cpi = &(cpv[i]);
// lbidList->UpdateMinMax(cpi->min, cpi->max, cpi->LBID, fColType.colDataType);
// }
// cpMutex.unlock(); //pthread_mutex_unlock(&cpMutex);
// cpv.clear();
// }
//
// mutex.lock(); //pthread_mutex_lock(&mutex);
// recvCount += msgCount;
// //...If producer is waiting, and we have gone below our threshold value,
// //...then we signal the producer to request more data from primproc
// if ( (sendWaiting) && ( (sentCount - recvCount) < fScanLbidReqThreshold ) )
// {
//#ifdef DEBUG2
// if (fOid >= 3000)
// cout << "pColScanStep consumer signaling producer for more data: "<<
// "st:" << fStepId <<
// "; sentCount-" << sentCount <<
// "; recvCount-" << recvCount <<
// "; threshold-" << fScanLbidReqThreshold << endl;
//#endif
// condvarWakeupProducer.notify_one(); //pthread_cond_signal(&condvarWakeupProducer);
// }
// } // end of loop to read LBID responses from primproc
//
// fPhysicalIO += l_physicalIO;
// fCacheIO += l_cachedIO;
// ridsReturned += l_ridsReturned;
//// cerr << "out of the main loop " << recvExited << endl;
// if (++recvExited == fNumThreads) {
// //...Casual partitioning could cause us to do no processing. In that
// //...case these time stamps did not get set. So we set them here.
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0) {
// dlTimes.setFirstReadTime();
// dlTimes.setLastReadTime();
// dlTimes.setFirstInsertTime();
// }
// if (fOid>=3000) dlTimes.setEndOfInputTime();
//
// //@bug 699: Reset StepMsgQueue
// fDec->removeQueue(uniqueID);
//
// if (fifo)
// fifo->endOfInput();
// else
// dlp->endOfInput();
// lastThread = true;
// }
//
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// if (fTableOid >= 3000 && lastThread)
// {
// //...Construct timestamp using ctime_r() instead of ctime() not
// //...necessarily due to re-entrancy, but because we want to strip
// //...the newline ('\n') off the end of the formatted string.
// time_t t = time(0);
// char timeString[50];
// ctime_r(&t, timeString);
// timeString[strlen(timeString)-1 ] = '\0';
//
// FifoDataList* pFifo = 0;
// uint64_t totalBlockedReadCount = 0;
// uint64_t totalBlockedWriteCount = 0;
//
// //...Sum up the blocked FIFO reads for all input associations
// size_t inDlCnt = fInputJobStepAssociation.outSize();
// for (size_t iDataList=0; iDataList<inDlCnt; iDataList++)
// {
// pFifo = fInputJobStepAssociation.outAt(iDataList)->fifoDL();
// if (pFifo)
// {
// totalBlockedReadCount += pFifo->blockedReadCount();
// }
// }
//
// //...Sum up the blocked FIFO writes for all output associations
// size_t outDlCnt = fOutputJobStepAssociation.outSize();
// for (size_t iDataList=0; iDataList<outDlCnt; iDataList++)
// {
// pFifo = fOutputJobStepAssociation.outAt(iDataList)->fifoDL();
// if (pFifo)
// {
// totalBlockedWriteCount += pFifo->blockedWriteCount();
// }
// }
//
// //...Roundoff inbound msg byte count to nearest KB for display;
// //...no need to do so for outbound, because it should be small.
// uint64_t msgBytesInKB = fMsgBytesIn >> 10;
// if (fMsgBytesIn & 512)
// msgBytesInKB++;
// // @bug 807
// if (fifo)
// fifo->totalSize(ridsReturned);
//
// if (traceOn())
// {
// //...Print job step completion information
// ostringstream logStr;
// logStr << "ses:" << fSessionId <<
// " st: " << fStepId << " finished at " <<
// timeString << "; PhyI/O-" << fPhysicalIO << "; CacheI/O-" <<
// fCacheIO << "; MsgsSent-" << fMsgsToPm << "; MsgsRcvd-" << recvCount <<
// "; BlockedFifoIn/Out-" << totalBlockedReadCount <<
// "/" << totalBlockedWriteCount <<
// "; output size-" << ridsReturned << endl <<
// "\tPartitionBlocksEliminated-" << fNumBlksSkipped <<
// "; MsgBytesIn-" << msgBytesInKB << "KB" <<
// "; MsgBytesOut-" << fMsgBytesOut << "B" << endl <<
// "\t1st read " << dlTimes.FirstReadTimeString() <<
// "; EOI " << dlTimes.EndOfInputTimeString() << "; runtime-" <<
// JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(),dlTimes.FirstReadTime()) <<
// "s" << endl;
//
// logEnd(logStr.str().c_str());
//
// syslogReadBlockCounts(16, // exemgr subsystem
// fPhysicalIO, // # blocks read from disk
// fCacheIO, // # blocks read from cache
// fNumBlksSkipped); // # casual partition block hits
// syslogProcessingTimes(16, // exemgr subsystem
// dlTimes.FirstReadTime(), // first datalist read
// dlTimes.LastReadTime(), // last datalist read
// dlTimes.FirstInsertTime(), // first datalist write
// dlTimes.EndOfInputTime()); // last (endOfInput) datalist write
// syslogEndStep(16, // exemgr subsystem
// totalBlockedReadCount, // blocked datalist input
// totalBlockedWriteCount, // blocked datalist output
// fMsgBytesIn, // incoming msg byte count
// fMsgBytesOut); // outgoing msg byte count
// }
//
// }
//
// if (fOid >=3000 && lastThread)
// lbidList->UpdateAllPartitionInfo();
//
//// cerr << "recv thread exiting" << endl;
}
void pColScanStep::addFilter(int8_t COP, float value) void pColScanStep::addFilter(int8_t COP, float value)
{ {
fFilterString << (uint8_t)COP; fFilterString << (uint8_t)COP;
@ -937,22 +223,6 @@ void pColScanStep::addFilter(int8_t COP, int64_t value, uint8_t roundFlag)
fFilterCount++; fFilterCount++;
} }
void pColScanStep::setBOP(int8_t B)
{
fBOP = B;
}
void pColScanStep::setSingleThread(bool b)
{
fSingleThread = b;
fNumThreads = 1;
}
void pColScanStep::setOutputType(int8_t OutputType)
{
fOutputType = OutputType;
}
const string pColScanStep::toString() const const string pColScanStep::toString() const
{ {
ostringstream oss; ostringstream oss;
@ -983,10 +253,7 @@ uint64_t pColScanStep::getFBO(uint64_t lbid)
{ {
lastLBID = extents[i].range.start + (extents[i].range.size << 10) - 1; lastLBID = extents[i].range.start + (extents[i].range.size << 10) - 1;
// lastLBID = extents[i].range.start + (extents[i].range.size * 1024) - 1;
// cerr << "start: " << extents[i].range.start << " end:" << lastLBID <<endl;
if (lbid >= (uint64_t)extents[i].range.start && lbid <= lastLBID) if (lbid >= (uint64_t)extents[i].range.start && lbid <= lastLBID)
// return (lbid - extents[i].range.start) + (extentSize * i);
return (lbid - extents[i].range.start) + (i << divShift); return (lbid - extents[i].range.start) + (i << divShift);
} }
@ -996,7 +263,6 @@ uint64_t pColScanStep::getFBO(uint64_t lbid)
pColScanStep::pColScanStep(const pColStep& rhs) : JobStep(rhs), fRm(rhs.resourceManager()), fMsgHeader() pColScanStep::pColScanStep(const pColStep& rhs) : JobStep(rhs), fRm(rhs.resourceManager()), fMsgHeader()
{ {
fNumThreads = fRm->getJlNumScanReceiveThreads();
fFilterCount = rhs.filterCount(); fFilterCount = rhs.filterCount();
fFilterString = rhs.filterString(); fFilterString = rhs.filterString();
isFilterFeeder = rhs.getFeederFlag(); isFilterFeeder = rhs.getFeederFlag();
@ -1005,14 +271,6 @@ pColScanStep::pColScanStep(const pColStep& rhs) : JobStep(rhs), fRm(rhs.resource
fColType = rhs.colType(); fColType = rhs.colType();
fBOP = rhs.BOP(); fBOP = rhs.BOP();
fIsDict = rhs.isDictCol(); fIsDict = rhs.isDictCol();
sentCount = 0;
recvCount = 0;
fScanLbidReqLimit = fRm->getJlScanLbidReqLimit();
fScanLbidReqThreshold = fRm->getJlScanLbidReqThreshold();
fStopSending = false;
fSingleThread = false;
fPhysicalIO = 0;
fCacheIO = 0;
fNumBlksSkipped = 0; fNumBlksSkipped = 0;
fMsgBytesIn = 0; fMsgBytesIn = 0;
fMsgBytesOut = 0; fMsgBytesOut = 0;
@ -1040,11 +298,6 @@ pColScanStep::pColScanStep(const pColStep& rhs) : JobStep(rhs), fRm(rhs.resource
numExtents = extents.size(); numExtents = extents.size();
extentSize = (fRm->getExtentRows() * fColType.colWidth) / BLOCK_SIZE; extentSize = (fRm->getExtentRows() * fColType.colWidth) / BLOCK_SIZE;
lbidList = rhs.lbidList; lbidList = rhs.lbidList;
// pthread_mutex_init(&mutex, NULL);
// pthread_mutex_init(&dlMutex, NULL);
// pthread_mutex_init(&cpMutex, NULL);
// pthread_cond_init(&condvar, NULL);
// pthread_cond_init(&condvarWakeupProducer, NULL);
finishedSending = sendWaiting = rDoNothing = false; finishedSending = sendWaiting = rDoNothing = false;
recvWaiting = 0; recvWaiting = 0;
recvExited = 0; recvExited = 0;
@ -1054,12 +307,7 @@ pColScanStep::pColScanStep(const pColStep& rhs) : JobStep(rhs), fRm(rhs.resource
rpbShift = rhs.rpbShift; rpbShift = rhs.rpbShift;
divShift = rhs.divShift; divShift = rhs.divShift;
// initializeConfigParms ( );
fTraceFlags = rhs.fTraceFlags; fTraceFlags = rhs.fTraceFlags;
// uniqueID = UniqueNumberGenerator::instance()->getUnique32();
// if (fDec)
// fDec->addQueue(uniqueID);
// fProducerThread = new SPTHD[fNumThreads];
} }
void pColScanStep::addFilters() void pColScanStep::addFilters()

View File

@ -58,47 +58,6 @@ using namespace BRM;
namespace joblist namespace joblist
{ {
#if 0
//const uint32_t defaultProjectBlockReqLimit = 32768;
//const uint32_t defaultProjectBlockReqThreshold = 16384;
struct pColStepPrimitive
{
pColStepPrimitive(pColStep* pColStep) : fPColStep(pColStep)
{}
pColStep* fPColStep;
void operator()()
{
try
{
fPColStep->sendPrimitiveMessages();
}
catch (exception& re)
{
cerr << "pColStep: send thread threw an exception: " << re.what() <<
"\t" << this << endl;
}
}
};
struct pColStepAggregator
{
pColStepAggregator(pColStep* pColStep) : fPColStepCol(pColStep)
{}
pColStep* fPColStepCol;
void operator()()
{
try
{
fPColStepCol->receivePrimitiveMessages();
}
catch (exception& re)
{
cerr << fPColStepCol->toString() << ": recv thread threw an exception: " << re.what() << endl;
}
}
};
#endif
pColStep::pColStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OID t, pColStep::pColStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OID t,
const CalpontSystemCatalog::ColType& ct, const JobInfo& jobInfo) const CalpontSystemCatalog::ColType& ct, const JobInfo& jobInfo)
: JobStep(jobInfo) : JobStep(jobInfo)
@ -117,14 +76,8 @@ pColStep::pColStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OID t,
, fIsDict(false) , fIsDict(false)
, isEM(jobInfo.isExeMgr) , isEM(jobInfo.isExeMgr)
, ridCount(0) , ridCount(0)
, fFlushInterval(jobInfo.flushInterval)
, fSwallowRows(false) , fSwallowRows(false)
, fProjectBlockReqLimit(fRm->getJlProjectBlockReqLimit())
, fProjectBlockReqThreshold(fRm->getJlProjectBlockReqThreshold())
, fStopSending(false)
, isFilterFeeder(false) , isFilterFeeder(false)
, fPhysicalIO(0)
, fCacheIO(0)
, fNumBlksSkipped(0) , fNumBlksSkipped(0)
, fMsgBytesIn(0) , fMsgBytesIn(0)
, fMsgBytesOut(0) , fMsgBytesOut(0)
@ -135,11 +88,6 @@ pColStep::pColStep(CalpontSystemCatalog::OID o, CalpontSystemCatalog::OID t,
int err, i; int err, i;
uint32_t mask; uint32_t mask;
if (fFlushInterval == 0 || !isEM)
fOutputType = OT_BOTH;
else
fOutputType = OT_TOKEN;
if (fOid < 1000) if (fOid < 1000)
throw runtime_error("pColStep: invalid column"); throw runtime_error("pColStep: invalid column");
@ -279,15 +227,7 @@ pColStep::pColStep(const pColScanStep& rhs)
, recvWaiting(false) , recvWaiting(false)
, fIsDict(rhs.isDictCol()) , fIsDict(rhs.isDictCol())
, ridCount(0) , ridCount(0)
,
// Per Cindy, it's save to put fFlushInterval to be 0
fFlushInterval(0)
, fSwallowRows(false) , fSwallowRows(false)
, fProjectBlockReqLimit(fRm->getJlProjectBlockReqLimit())
, fProjectBlockReqThreshold(fRm->getJlProjectBlockReqThreshold())
, fStopSending(false)
, fPhysicalIO(0)
, fCacheIO(0)
, fNumBlksSkipped(0) , fNumBlksSkipped(0)
, fMsgBytesIn(0) , fMsgBytesIn(0)
, fMsgBytesOut(0) , fMsgBytesOut(0)
@ -390,15 +330,7 @@ pColStep::pColStep(const PassThruStep& rhs)
, recvWaiting(false) , recvWaiting(false)
, fIsDict(rhs.isDictCol()) , fIsDict(rhs.isDictCol())
, ridCount(0) , ridCount(0)
,
// Per Cindy, it's save to put fFlushInterval to be 0
fFlushInterval(0)
, fSwallowRows(false) , fSwallowRows(false)
, fProjectBlockReqLimit(fRm->getJlProjectBlockReqLimit())
, fProjectBlockReqThreshold(fRm->getJlProjectBlockReqThreshold())
, fStopSending(false)
, fPhysicalIO(0)
, fCacheIO(0)
, fNumBlksSkipped(0) , fNumBlksSkipped(0)
, fMsgBytesIn(0) , fMsgBytesIn(0)
, fMsgBytesOut(0) , fMsgBytesOut(0)
@ -474,107 +406,6 @@ pColStep::pColStep(const PassThruStep& rhs)
sort(extents.begin(), extents.end(), ExtentSorter()); sort(extents.begin(), extents.end(), ExtentSorter());
numExtents = extents.size(); numExtents = extents.size();
// uniqueID = UniqueNumberGenerator::instance()->getUnique32();
// if (fDec)
// fDec->addQueue(uniqueID);
// initializeConfigParms ( );
}
pColStep::~pColStep()
{
// join?
// delete lbidList;
// if (fDec)
// fDec->removeQueue(uniqueID);
}
//------------------------------------------------------------------------------
// Initialize configurable parameters
//------------------------------------------------------------------------------
void pColStep::initializeConfigParms()
{
// const string section ( "JobList" );
// const string sendLimitName ( "ProjectBlockReqLimit" );
// const string sendThresholdName ( "ProjectBlockReqThreshold" );
// Config* cf = Config::makeConfig();
//
// string strVal;
// uint64_t numVal;
//...Get the tuning parameters that throttle msgs sent to primproc
//...fFilterRowReqLimit puts a cap on how many rids we will request from
//... primproc, before pausing to let the consumer thread catch up.
//... Without this limit, there is a chance that PrimProc could flood
//... ExeMgr with thousands of messages that will consume massive
//... amounts of memory for a 100 gigabyte database.
//...fFilterRowReqThreshhold is the level at which the number of outstanding
//... rids must fall below, before the producer can send more rids.
// strVal = cf->getConfig(section, sendLimitName);
// if (strVal.size() > 0)
// {
// errno = 0;
// numVal = Config::uFromText(strVal);
// if ( errno == 0 )
// fProjectBlockReqLimit = (uint32_t)numVal;
// }
//
// strVal = cf->getConfig(section, sendThresholdName);
// if (strVal.size() > 0)
// {
// errno = 0;
// numVal = Config::uFromText(strVal);
// if ( errno == 0 )
// fProjectBlockReqThreshold = (uint32_t)numVal;
// }
}
void pColStep::startPrimitiveThread()
{
// pThread.reset(new boost::thread(pColStepPrimitive(this)));
}
void pColStep::startAggregationThread()
{
// cThread.reset(new boost::thread(pColStepAggregator(this)));
}
void pColStep::run()
{
// if (traceOn())
// {
// syslogStartStep(16, // exemgr subsystem
// std::string("pColStep")); // step name
// }
//
// size_t sz = fInputJobStepAssociation.outSize();
// idbassert(sz > 0);
// const AnyDataListSPtr& dl = fInputJobStepAssociation.outAt(0);
// DataList_t* dlp = dl->dataList();
// DataList<StringElementType>* strDlp = dl->stringDataList();
// if ( dlp )
// setRidList(dlp);
// else
// {
// setStrRidList( strDlp );
// }
// //Sort can be set through the jobstep or the input JSA if fFlushinterval is 0
// fToSort = (fFlushInterval) ? 0 : (!fToSort) ? fInputJobStepAssociation.toSort() : fToSort;
// fToSort = 0;
// //pthread_mutex_init(&mutex, NULL);
// //pthread_cond_init(&condvar, NULL);
// //pthread_cond_init(&flushed, NULL);
// startPrimitiveThread();
// startAggregationThread();
}
void pColStep::join()
{
// pThread->join();
// cThread->join();
// //pthread_mutex_destroy(&mutex);
// //pthread_cond_destroy(&condvar);
// //pthread_cond_destroy(&flushed);
} }
void pColStep::addFilter(int8_t COP, float value) void pColStep::addFilter(int8_t COP, float value)
@ -636,732 +467,6 @@ void pColStep::addFilter(int8_t COP, const int128_t& value, uint8_t roundFlag)
fFilterCount++; fFilterCount++;
} }
void pColStep::setRidList(DataList<ElementType>* dl)
{
ridList = dl;
}
void pColStep::setStrRidList(DataList<StringElementType>* strDl)
{
strRidList = strDl;
}
void pColStep::setBOP(int8_t b)
{
fBOP = b;
}
void pColStep::setOutputType(int8_t OutputType)
{
fOutputType = OutputType;
}
void pColStep::setSwallowRows(const bool swallowRows)
{
fSwallowRows = swallowRows;
}
void pColStep::sendPrimitiveMessages()
{
// int it = -1;
// int msgRidCount = 0;
// int ridListIdx = 0;
// bool more = false;
// uint64_t absoluteRID = 0;
// int64_t msgLBID = -1;
// int64_t nextLBID = -1;
// int64_t msgLargeBlock = -1;
// int64_t nextLargeBlock = -1;
// uint16_t blockRelativeRID;
// uint32_t msgCount = 0;
// uint32_t sentBlockCount = 0;
// int msgsSkip=0;
// bool scan=false;
// bool scanThisBlock=false;
// ElementType e;
// UintRowGroup rw;
// StringElementType strE;
// StringRowGroup strRw;
//
// ByteStream msgRidList;
// ByteStream primMsg(MAX_BUFFER_SIZE); //the MAX_BUFFER_SIZE as of 8/20
//
// NewColRequestHeader hdr;
//
// AnyDataListSPtr dl;
// FifoDataList *fifo = NULL;
// StringFifoDataList* strFifo = NULL;
//
// const bool ignoreCP = ((fTraceFlags & CalpontSelectExecutionPlan::IGNORE_CP) != 0);
//
// //The presence of more than 1 input DL means we (probably) have a pDictionaryScan step feeding this
//step
// // a list of tokens to get the rids for. Convert the input tokens to a filter string. We also have a
//rid
// // list as the second input dl
// if (fInputJobStepAssociation.outSize() > 1)
// {
// addFilters();
// if (fTableOid >= 3000)
// cout << toString() << endl;
// //If we got no input rids (as opposed to no input DL at all) then there were no matching rows
//from
// // the previous step, so this step should not return any rows either. This would be the case,
//for
// // instance, if P_NAME LIKE '%xxxx%' produced no signature matches.
// if (fFilterCount == 0)
// {
// goto done;
// }
// }
//
// // determine which ranges/extents to eliminate from this step
//
//#ifdef DEBUG
// if (fOid>=3000)
// cout << "oid " << fOid << endl;
//#endif
//
// scanFlags.resize(numExtents);
//
// for (uint32_t idx=0; idx <numExtents; idx++)
// {
// if (extents[idx].partition.cprange.isValid != BRM::CP_VALID) {
// scanFlags[idx]=1;
// }
// else
// {
//
// bool flag = lbidList->CasualPartitionPredicate(
// extents[idx].partition.cprange.loVal,
// extents[idx].partition.cprange.hiVal,
// &fFilterString,
// fFilterCount,
// fColType,
// fBOP) || ignoreCP;
// scanFlags[idx]=flag;
//#ifdef DEBUG
// if (fOid >= 3000 && flushInterval == 0)
// cout << (flag ? " will scan " : " will not scan ")
// << "extent with range " << extents[idx].partition.cprange.loVal
// << "-" << extents[idx].partition.cprange.hiVal << endl;
//#endif
//
// }
//
//// if (fOid>=3000)
//// cout << " " << scanFlags[idx];
// }
//// if (scanFlags.size()>0)
//// cout << endl;
//
// // If there was more than 1 input DL, the first is a list of filters and the second is a list of rids,
// // otherwise the first is the list of rids.
// if (fInputJobStepAssociation.outSize() > 1)
// ridListIdx = 1;
// else
// ridListIdx = 0;
//
// dl = fInputJobStepAssociation.outAt(ridListIdx);
// ridList = dl->dataList();
// if ( ridList )
// {
// fifo = dl->fifoDL();
//
// if (fifo)
// it = fifo->getIterator();
// else
// it = ridList->getIterator();
// }
// else
// {
// strRidList = dl->stringDataList();
// strFifo = dl->stringDL();
//
// if (strFifo)
// it = strFifo->getIterator();
// else
// it = strRidList->getIterator();
// }
//
// if (ridList)
// {
// if (fifo)
// {
// more = fifo->next(it, &rw);
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0) {
// dlTimes.setFirstReadTime();
// }
// absoluteRID = rw.et[0].first;
// }
// else
// {
// more = ridList->next(it, &e);
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0) {
// dlTimes.setFirstReadTime();
// }
// absoluteRID = e.first;
// rw.count = 1;
// }
// }
// else
// {
// if (strFifo)
// {
// more = strFifo->next(it, &strRw);
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0) {
// dlTimes.setFirstReadTime();
// }
// absoluteRID = strRw.et[0].first;
// }
// else
// {
// more = strRidList->next(it, &strE);
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0) {
// dlTimes.setFirstReadTime();
// }
// absoluteRID = strE.first;
// strRw.count = 1;
// }
// }
//
// if (more)
// msgLBID = getLBID(absoluteRID, scan);
// scanThisBlock = scan;
// msgLargeBlock = absoluteRID >> blockSizeShift;
//
// while (more || msgRidCount > 0) {
// uint64_t rwCount;
// if ( ridList)
// rwCount = rw.count;
// else
// rwCount = strRw.count;
//
// for (uint64_t i = 0; ((i < rwCount) || (!more && msgRidCount > 0)); )
// {
// if ( ridList)
// {
// if (fifo)
// absoluteRID = rw.et[i].first;
// else
// absoluteRID = e.first;
// }
// else
// {
// if (strFifo)
// absoluteRID = strRw.et[i].first;
// else
// absoluteRID = strE.first;
// }
//
// if (more) {
// nextLBID = getLBID(absoluteRID, scan);
// nextLargeBlock = absoluteRID >> blockSizeShift;
// }
//
// //XXXPAT: need to prove N & S here
// if (nextLBID == msgLBID && more) {
//// blockRelativeRID = absoluteRID % ridsPerBlock;
// blockRelativeRID = absoluteRID & rpbMask;
// msgRidList << blockRelativeRID;
// msgRidCount++;
// ++i;
// }
// else {
// //Bug 831: move building msg after the check of scanThisBlock
// if (scanThisBlock==true)
// {
// hdr.ism.Interleave=0;
// hdr.ism.Flags=planFlagsToPrimFlags(fTraceFlags);
// hdr.ism.Command=COL_BY_SCAN;
// hdr.ism.Size=sizeof(NewColRequestHeader) + fFilterString.length() +
// msgRidList.length();
// hdr.ism.Type=2;
//
// hdr.hdr.SessionID = fSessionId;
// //hdr.hdr.StatementID = 0;
// hdr.hdr.TransactionID = fTxnId;
// hdr.hdr.VerID = fVerId;
// hdr.hdr.StepID = fStepId;
// hdr.hdr.UniqueID = uniqueID;
//
// hdr.LBID = msgLBID;
//// idbassert(hdr.LBID >= 0);
// hdr.DataSize = fColType.colWidth;
// hdr.DataType = fColType.colDataType;
// hdr.CompType = fColType.compressionType;
// hdr.OutputType = fOutputType;
// hdr.BOP = fBOP;
// hdr.NOPS = fFilterCount;
// hdr.NVALS = msgRidCount;
// hdr.sort = fToSort;
//
// primMsg.append((const uint8_t *) &hdr, sizeof(NewColRequestHeader));
// primMsg += fFilterString;
// primMsg += msgRidList;
// ridCount += msgRidCount;
// ++sentBlockCount;
//
//#ifdef DEBUG
// if (flushInterval == 0 && fOid >= 3000)
// cout << "sending a prim msg for LBID " << msgLBID << endl;
//#endif
// ++msgCount;
//// cout << "made a primitive\n";
// if (msgLargeBlock != nextLargeBlock || !more) {
//// cout << "writing " << msgCount << " primitives\n";
// fMsgBytesOut += primMsg.lengthWithHdrOverhead();
// fDec->write(primMsg);
// msgsSent += msgCount;
// msgCount = 0;
// primMsg.restart();
// msgLargeBlock = nextLargeBlock;
//
// // @bug 769 - Added "&& !fSwallowRows" condition below to fix problem
//with
// // caltraceon(16) not working for tpch01 and some other queries. If a
//query
// // ever held off requesting more blocks, it would lock and never finish.
// //Bug 815
// if (( sentBlockCount >= fProjectBlockReqLimit) && !fSwallowRows
//&&
// (( msgsSent - msgsRecvd) > fProjectBlockReqThreshold))
// {
// mutex.lock(); //pthread_mutex_lock(&mutex);
// fStopSending = true;
//
// // @bug 836. Wake up the receiver if he's sleeping.
// if (recvWaiting)
// condvar.notify_one();
////pthread_cond_signal(&condvar); flushed.wait(mutex); //pthread_cond_wait(&flushed, &mutex); fStopSending
//= false; mutex.unlock(); //pthread_mutex_unlock(&mutex); sentBlockCount = 0;
// }
// }
// }
// else
// {
// msgsSkip++;
// }
// msgLBID = nextLBID;
// msgRidList.restart();
// msgRidCount = 0;
//
// mutex.lock(); //pthread_mutex_lock(&mutex);
//
// if (scanThisBlock) {
// if (recvWaiting)
// condvar.notify_one(); //pthread_cond_signal(&condvar);
// #ifdef DEBUG
//// cout << "msgsSent++ = " << msgsSent << endl;
// #endif
// }
// scanThisBlock = scan;
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// // break the for loop
// if (!more)
// break;
// }
// } // for rw.count
//
// if (more)
// {
// if ( ridList )
// {
// if (fifo)
// {
// rw.count = 0;
// more = fifo->next(it, &rw);
// }
// else
// {
// rw.count = 1;
// more = ridList->next(it, &e);
// }
// }
// else
// {
// if (strFifo)
// {
// strRw.count = 0;
// more = strFifo->next(it, &strRw);
// }
// else
// {
// strRw.count = 1;
// more = strRidList->next(it, &strE);
// }
// }
// }
// }
//
// if (fOid>=3000) dlTimes.setLastReadTime();
//
// done:
// mutex.lock(); //pthread_mutex_lock(&mutex);
// finishedSending = true;
// if (recvWaiting)
// condvar.notify_one(); //pthread_cond_signal(&condvar);
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
//#ifdef DEBUG
// if (fOid >=3000)
// cout << "pColStep msgSent "
// << msgsSent << "/" << msgsSkip
// << " rids " << ridCount
// << " oid " << fOid << " " << msgLBID << endl;
//#endif
// //...Track the number of LBIDs we skip due to Casual Partioning.
// fNumBlksSkipped += msgsSkip;
}
void pColStep::receivePrimitiveMessages()
{
// int64_t ridResults = 0;
// AnyDataListSPtr dl = fOutputJobStepAssociation.outAt(0);
// DataList_t* dlp = dl->dataList();
// uint64_t fbo;
// FifoDataList *fifo = dl->fifoDL();
// UintRowGroup rw;
// uint64_t ridBase;
// boost::shared_ptr<ByteStream> bs;
// uint32_t i = 0, length;
//
// while (1) {
// // sync with the send side
// mutex.lock(); //pthread_mutex_lock(&mutex);
// while (!finishedSending && msgsSent == msgsRecvd) {
// recvWaiting = true;
// #ifdef DEBUG
// cout << "c sleeping" << endl;
// #endif
// // @bug 836. Wake up the sender if he's sleeping.
// if (fStopSending)
// flushed.notify_one(); //pthread_cond_signal(&flushed);
// condvar.wait(mutex); //pthread_cond_wait(&condvar, &mutex);
// #ifdef DEBUG
// cout << "c waking" << endl;
// #endif
// recvWaiting = false;
// }
// if (msgsSent == msgsRecvd) {
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
// break;
// }
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// // do the recv
// fDec->read(uniqueID, bs);
// fMsgBytesIn += bs->lengthWithHdrOverhead();
//
// // no more messages, and buffered messages should be already processed by now.
// if (bs->length() == 0) break;
//
// #ifdef DEBUG
// cout << "msgsRecvd++ = " << msgsRecvd << ". RidResults = " << ridResults << endl;
// cout << "Got a ColResultHeader!: " << bs.length() << " bytes" << endl;
// #endif
//
// const ByteStream::byte* bsp = bs->buf();
//
// // get the ISMPacketHeader out of the bytestream
// //const ISMPacketHeader* ism = reinterpret_cast<const ISMPacketHeader*>(bsp);
//
// // get the ColumnResultHeader out of the bytestream
// const ColResultHeader* crh = reinterpret_cast<const ColResultHeader*>
// (&bsp[sizeof(ISMPacketHeader)]);
//
// bool firstRead = true;
// length = bs->length();
//
// i = 0;
// uint32_t msgCount = 0;
// while (i < length) {
// ++msgCount;
//
// i += sizeof(ISMPacketHeader);
// crh = reinterpret_cast<const ColResultHeader*>(&bsp[i]);
// // double check the sequence number is increased by one each time
// i += sizeof(ColResultHeader);
//
// fCacheIO += crh->CacheIO;
// fPhysicalIO += crh->PhysicalIO;
//
// // From this point on the rest of the bytestream is the data that comes back from the
//primitive server
// // This needs to be fed to a datalist that is retrieved from the outputassociation
//object.
//
// fbo = getFBO(crh->LBID);
// ridBase = fbo << rpbShift;
//
// #ifdef DEBUG
//// cout << " NVALS = " << crh->NVALS << " fbo = " << fbo << " lbid = " << crh->LBID <<
///endl;
// #endif
//
// //Check output type
// if ( fOutputType == OT_RID )
// {
// ridResults += crh->NVALS;
// }
//
// /* XXXPAT: This clause is executed when ExeMgr calls the new nextBand(BS) fcn.
//
// TODO: both classes have to agree
// on which nextBand() variant will be called. pColStep
// currently has to infer that from flushInterval and the
// Table OID. It would be better to have a more explicit form
// of agreement.
//
// The goal of the nextBand(BS) fcn is to avoid iterating over
// every row except at unserialization. This clause copies
// the raw results from the PrimProc response directly into
// the memory used for the ElementType array. DeliveryStep
// will also treat the ElementType array as raw memory and
// serialize that. TableColumn now parses the packed data
// instead of whole ElementTypes.
// */
// else if (fOutputType == OT_TOKEN && fFlushInterval > 0 && !fIsDict) {
//
// if (fOid>=3000 && dlTimes.FirstInsertTime().tv_sec==0)
// dlTimes.setFirstInsertTime();
// ridResults += crh->NVALS;
//
// /* memcpy the bytestream into the output set */
// uint32_t toCopy, bsPos = 0;
// uint8_t *pos;
// while (bsPos < crh->NVALS) {
// toCopy = (crh->NVALS - bsPos > rw.ElementsPerGroup - rw.count ?
// rw.ElementsPerGroup - rw.count : crh->NVALS - bsPos);
// pos = ((uint8_t *) &rw.et[0]) + (rw.count * fColType.colWidth);
// memcpy(pos, &bsp[i], toCopy * fColType.colWidth);
// bsPos += toCopy;
// i += toCopy * fColType.colWidth;
// rw.count += toCopy;
// if (rw.count == rw.ElementsPerGroup) {
// if (!fSwallowRows)
// fifo->insert(rw);
// rw.count = 0;
// }
// }
// }
// else if ( fOutputType == OT_TOKEN)
// {
// uint64_t dv;
// uint64_t rid;
//
// if (fOid>=3000 && dlTimes.FirstInsertTime().tv_sec==0)
// dlTimes.setFirstInsertTime();
// ridResults += crh->NVALS;
// for(int j = 0; j < crh->NVALS; ++j)
// {
// // XXXPAT: Only use this when the RID doesn't matter or when
// // the response contains every row.
//
// rid = j + ridBase;
// switch (fColType.colWidth) {
// case 8: dv = *((const uint64_t *) &bsp[i]); i += 8; break;
// case 4: dv = *((const uint32_t *) &bsp[i]); i += 4; break;
// case 2: dv = *((const uint16_t *) &bsp[i]); i += 2; break;
// case 1: dv = *((const uint8_t *) &bsp[i]); ++i; break;
// default:
// throw runtime_error("pColStep: invalid column
//width!");
// }
//
// // @bug 663 - Don't output any rows if fSwallowRows (caltraceon(16)) is
//on.
// // This options swallows rows in the project steps.
// if (!fSwallowRows)
// {
// if (fifo)
// {
// rw.et[rw.count].first = rid;
// rw.et[rw.count++].second = dv;
// if (rw.count == rw.ElementsPerGroup)
// {
// fifo->insert(rw);
// rw.count = 0;
// }
// }
// else
// {
// dlp->insert(ElementType(rid, dv));
// }
// #ifdef DEBUG
// //cout << " -- inserting <" << rid << ", " << dv << "> " << *prid <<
//endl; #endif
// }
// }
// }
// else if ( fOutputType == OT_BOTH )
// {
// ridResults += crh->NVALS;
// for(int j = 0; j < crh->NVALS; ++j)
// {
// uint64_t dv;
// uint64_t rid;
//
// rid = *((const uint16_t *) &bsp[i]) + ridBase;
// i += sizeof(uint16_t);
// switch (fColType.colWidth) {
// case 8: dv = *((const uint64_t *) &bsp[i]); i += 8;
//break; case 4: dv = *((const uint32_t *) &bsp[i]); i += 4; break; case 2: dv = *((const uint16_t *)
//&bsp[i]); i += 2; break; case 1: dv = *((const uint8_t *) &bsp[i]); ++i; break; default: throw
//runtime_error("pColStep: invalid column width!");
// }
//
// // @bug 663 - Don't output any rows if fSwallowRows (caltraceon(16)) is
//on.
// // This options swallows rows in the project steps.
// if (!fSwallowRows) {
// if (fOid>=3000 && dlTimes.FirstInsertTime().tv_sec==0)
// dlTimes.setFirstInsertTime();
// if(fifo)
// {
//// rw.et[rw.count++] = ElementType(rid, dv);
// rw.et[rw.count].first = rid;
// rw.et[rw.count++].second = dv;
// if (rw.count == rw.ElementsPerGroup)
// {
// fifo->insert(rw);
// rw.count = 0;
// }
// }
// else
// {
// dlp->insert(ElementType(rid, dv));
// }
// #ifdef DEBUG
// //cout << " -- inserting <" << rid << ", " << dv << "> " << *prid <<
//endl; #endif
// }
// }
// }
// } // unpacking the BS
//
// //Bug 815: Check whether we have enough to process
// //++lockCount;
// mutex.lock(); //pthread_mutex_lock(&mutex);
// if ( fStopSending && ((msgsSent - msgsRecvd ) <= fProjectBlockReqThreshold) )
// {
// flushed.notify_one(); //pthread_cond_signal(&flushed);
// }
// mutex.unlock(); //pthread_mutex_unlock(&mutex);
//
// firstRead = false;
// msgsRecvd += msgCount;
// } // read loop
// // done reading
//
// if (fifo && rw.count > 0)
// fifo->insert(rw);
//
// //...Casual partitioning could cause us to do no processing. In that
// //...case these time stamps did not get set. So we set them here.
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0) {
// dlTimes.setFirstReadTime();
// dlTimes.setLastReadTime();
// dlTimes.setFirstInsertTime();
// }
// if (fOid>=3000) dlTimes.setEndOfInputTime();
//
// //@bug 699: Reset StepMsgQueue
// fDec->removeQueue(uniqueID);
//
// if (fifo)
// fifo->endOfInput();
// else
// dlp->endOfInput();
//
// if (fTableOid >= 3000)
// {
// //...Construct timestamp using ctime_r() instead of ctime() not
// //...necessarily due to re-entrancy, but because we want to strip
// //...the newline ('\n') off the end of the formatted string.
// time_t t = time(0);
// char timeString[50];
// ctime_r(&t, timeString);
// timeString[strlen(timeString)-1 ] = '\0';
//
// FifoDataList* pFifo = 0;
// uint64_t totalBlockedReadCount = 0;
// uint64_t totalBlockedWriteCount = 0;
//
// //...Sum up the blocked FIFO reads for all input associations
// size_t inDlCnt = fInputJobStepAssociation.outSize();
// for (size_t iDataList=0; iDataList<inDlCnt; iDataList++)
// {
// pFifo = fInputJobStepAssociation.outAt(iDataList)->fifoDL();
// if (pFifo)
// {
// totalBlockedReadCount += pFifo->blockedReadCount();
// }
// }
//
// //...Sum up the blocked FIFO writes for all output associations
// size_t outDlCnt = fOutputJobStepAssociation.outSize();
// for (size_t iDataList=0; iDataList<outDlCnt; iDataList++)
// {
// pFifo = fOutputJobStepAssociation.outAt(iDataList)->fifoDL();
// if (pFifo)
// {
// totalBlockedWriteCount += pFifo->blockedWriteCount();
// }
// }
//
// //...Roundoff msg byte counts to nearest KB for display
// uint64_t msgBytesInKB = fMsgBytesIn >> 10;
// uint64_t msgBytesOutKB = fMsgBytesOut >> 10;
// if (fMsgBytesIn & 512)
// msgBytesInKB++;
// if (fMsgBytesOut & 512)
// msgBytesOutKB++;
//
// // @bug 828
// if (fifo)
// fifo->totalSize(ridResults);
//
// if (traceOn())
// {
// //...Print job step completion information
// ostringstream logStr;
// logStr << "ses:" << fSessionId <<
// " st: " << fStepId << " finished at " <<
// timeString << "; PhyI/O-" << fPhysicalIO << "; CacheI/O-" <<
// fCacheIO << "; MsgsRvcd-" << msgsRecvd <<
// "; BlockedFifoIn/Out-" << totalBlockedReadCount <<
// "/" << totalBlockedWriteCount <<
// "; output size-" << ridResults << endl <<
// "\tPartitionBlocksEliminated-" << fNumBlksSkipped <<
// "; MsgBytesIn-" << msgBytesInKB << "KB" <<
// "; MsgBytesOut-" << msgBytesOutKB << "KB" << endl <<
// "\t1st read " << dlTimes.FirstReadTimeString() <<
// "; EOI " << dlTimes.EndOfInputTimeString() << "; runtime-" <<
// JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(),dlTimes.FirstReadTime()) <<
// "s" << endl;
//
// logEnd(logStr.str().c_str());
//
// syslogReadBlockCounts(16, // exemgr sybsystem
// fPhysicalIO, // # blocks read from disk
// fCacheIO, // # blocks read from cache
// fNumBlksSkipped); // # casual partition block hits
// syslogProcessingTimes(16, // exemgr subsystem
// dlTimes.FirstReadTime(), // first datalist read
// dlTimes.LastReadTime(), // last datalist read
// dlTimes.FirstInsertTime(), // first datalist write
// dlTimes.EndOfInputTime()); // last (endOfInput) datalist write
// syslogEndStep(16, // exemgr subsystem
// totalBlockedReadCount, // blocked datalist input
// totalBlockedWriteCount, // blocked datalist output
// fMsgBytesIn, // incoming msg byte count
// fMsgBytesOut); // outgoing msg byte count
// }
// }
}
const string pColStep::toString() const const string pColStep::toString() const
{ {
ostringstream oss; ostringstream oss;

View File

@ -105,64 +105,11 @@ pDictionaryStep::pDictionaryStep(CalpontSystemCatalog::OID o, CalpontSystemCatal
, fFilterCount(0) , fFilterCount(0)
, requestList(0) , requestList(0)
, fInterval(jobInfo.flushInterval) , fInterval(jobInfo.flushInterval)
, fPhysicalIO(0)
, fCacheIO(0)
, fMsgBytesIn(0) , fMsgBytesIn(0)
, fMsgBytesOut(0) , fMsgBytesOut(0)
, fRm(jobInfo.rm) , fRm(jobInfo.rm)
, hasEqualityFilter(false) , hasEqualityFilter(false)
{ {
// uniqueID = UniqueNumberGenerator::instance()->getUnique32();
// fColType.compressionType = fColType.ddn.compressionType = ct;
}
pDictionaryStep::~pDictionaryStep()
{
// if (fDec)
// fDec->removeQueue(uniqueID);
}
void pDictionaryStep::startPrimitiveThread()
{
// pThread.reset(new boost::thread(pDictionaryStepPrimitive(this)));
}
void pDictionaryStep::startAggregationThread()
{
// cThread.reset(new boost::thread(pDictStepAggregator(this)));
}
void pDictionaryStep::run()
{
// if (traceOn())
// {
// syslogStartStep(16, // exemgr subsystem
// std::string("pDictionaryStep")); // step name
// }
//
// const AnyDataListSPtr& dl = fInputJobStepAssociation.outAt(0);
// DataList_t* dlp = dl->dataList();
// setInputList(dlp);
//
// startPrimitiveThread();
// startAggregationThread();
}
void pDictionaryStep::join()
{
// pThread->join();
// cThread->join();
}
void pDictionaryStep::setInputList(DataList_t* dl)
{
requestList = dl;
}
void pDictionaryStep::setBOP(int8_t b)
{
fBOP = b;
} }
void pDictionaryStep::addFilter(int8_t COP, const string& value) void pDictionaryStep::addFilter(int8_t COP, const string& value)
@ -190,322 +137,6 @@ void pDictionaryStep::addFilter(int8_t COP, const string& value)
} }
} }
void pDictionaryStep::sendPrimitiveMessages()
{
// int it = -1;
// int msgRidCount = 0;
// bool more;
// int64_t sigToken, msgLBID, nextLBID = -1;
// uint16_t sigOrd;
// ByteStream msgRidList, primMsg(65536); //the MAX_BUFFER_SIZE as of 8/20
// DictSignatureRequestHeader hdr;
// ISMPacketHeader ism;
// OldGetSigParams pt;
// FifoDataList* fifo = fInputJobStepAssociation.outAt(0)->fifoDL();
// UintRowGroup rw;
//
///* XXXPAT: Does this primitive need to care about the HWM as a sanity check, given
// that a ridlist is supplied? */
//
// if (fifo == 0)
// throw logic_error("Use p_colscanrange instead here");
//
// try{
// it = fifo->getIterator();
// }catch(exception& ex) {
// cerr << "pDictionaryStep::sendPrimitiveMessages: caught exception: " << ex.what() << endl;
// }catch(...) {
// cerr << "pDictionaryStep::sendPrimitiveMessages: caught exception" << endl;
// }
//
// more = fifo->next(it, &rw);
//
// sigToken = rw.et[0].second;
// msgLBID = sigToken >> 10;
// while (more || msgRidCount > 0) {
// for (uint64_t i = 0; ((i < rw.count) || (!more && msgRidCount > 0)); )
// {
// if (more)
// {
// ridCount++;
// sigToken = rw.et[i].second;
// nextLBID = sigToken >> 10;
//#ifdef DEBUG
// cout << "sigToken = " << sigToken << " lbid = " << nextLBID << endl;
//#endif
// }
//
// // @bug 472
// if (nextLBID == msgLBID && more && msgRidCount < 8000) { //XXXPAT: need to prove N & S
//here sigOrd = sigToken & 0x3ff; pt.rid = (nextLBID >= 0 ? rw.et[i].first : 0x8000000000000000LL |
//rw.et[i].first); pt.offsetIndex = sigOrd; msgRidList.append(reinterpret_cast<ByteStream::byte*>(&pt),
//sizeof(pt)); msgRidCount++;
// ++i;
//#ifdef DEBUG
// cout << "added signature ordinal " << sigOrd << endl;
//#endif
// }
// else {
//#ifdef DEBUG
// cout << "sending a prim msg" << endl;
//#endif
//
// // send the primitive, start constructing the next msg
// ism.Interleave=0;
// ism.Flags=planFlagsToPrimFlags(fTraceFlags);
// ism.Command=DICT_SIGNATURE;
// ism.Size=sizeof(DictSignatureRequestHeader) + msgRidList.length();
// ism.Type=2;
//
// hdr.Hdr.SessionID = fSessionId;
// //hdr.Hdr.StatementID = 0;
// hdr.Hdr.TransactionID = fTxnId;
// hdr.Hdr.VerID = fVerId;
// hdr.Hdr.StepID = fStepId;
// hdr.Hdr.UniqueID = uniqueID;
//
// hdr.LBID = msgLBID;
// idbassert(msgRidCount <= 8000);
// hdr.NVALS = msgRidCount;
// hdr.CompType = fColType.ddn.compressionType;
//
// primMsg.load((const uint8_t *) &ism, sizeof(ism));
// primMsg.append((const uint8_t *) &hdr, sizeof(DictSignatureRequestHeader));
// primMsg += msgRidList;
// fMsgBytesOut += primMsg.lengthWithHdrOverhead();
// fDec->write(primMsg);
//
// msgLBID = nextLBID;
// primMsg.restart();
// msgRidList.restart();
// msgRidCount = 0;
//
// mutex.lock();
// msgsSent++;
// if (recvWaiting)
// condvar.notify_one();
//#ifdef DEBUG
// cout << "msgsSent++ = " << msgsSent << endl;
//#endif
// mutex.unlock();
//
// if (!more)
// break;
// }
// } // rw.count
//
// if (more)
// {
// rw.count = 0;
// more = fifo->next(it, &rw);
// }
// }
//
// mutex.lock();
// finishedSending = true;
// if (recvWaiting)
// condvar.notify_one();
// mutex.unlock();
}
void pDictionaryStep::receivePrimitiveMessages()
{
// int64_t ridResults = 0;
// AnyDataListSPtr dl = fOutputJobStepAssociation.outAt(0);
// StrDataList* dlp = dl->stringDataList();
// StringFifoDataList *fifo = fOutputJobStepAssociation.outAt(0)->stringDL();
// StringRowGroup rw;
//
// while (1) {
//
// // sync with the send side
// mutex.lock();
//
// while (!finishedSending && msgsSent==msgsRecvd) {
// recvWaiting = true;
// condvar.wait(mutex);
// if (msgsSent == msgsRecvd) {
// mutex.unlock();
// break;
// }
// recvWaiting = false;
// }
//
// if (finishedSending != 0 && msgsRecvd >= msgsSent) {
// goto junk;
// }
// mutex.unlock();
//
// // do the recv
//
// ByteStream bs = fDec->read(uniqueID);
// fMsgBytesIn += bs.lengthWithHdrOverhead();
// if (fOid>=3000 && dlTimes.FirstReadTime().tv_sec==0)
// dlTimes.setFirstReadTime();
// if (fOid>=3000) dlTimes.setLastReadTime();
//
// msgsRecvd++;
// if (bs.length() == 0)
// break;
//
// const ByteStream::byte* bsp = bs.buf();
//
// // get the ResultHeader out of the bytestream
// const DictOutput* drh = reinterpret_cast<const DictOutput*>(bsp);
//
// bsp += sizeof(DictOutput);
//
// fCacheIO += drh->CacheIO;
// fPhysicalIO += drh->PhysicalIO;
//
// // From this point on the rest of the bytestream is the data that comes back from the primitive
//server
// // This needs to be fed to a datalist that is retrieved from the outputassociation object.
//
// char d[8192];
//// memset(d, 0, 8192);
// if (fOid>=3000 && dlTimes.FirstInsertTime().tv_sec==0)
// dlTimes.setFirstInsertTime();
// for(int j = 0; j < drh->NVALS; j++)
// {
// const uint64_t* ridp = (const uint64_t*)bsp;
// bsp += sizeof(*ridp);
// uint64_t rid = *ridp;
// const uint16_t* lenp = (const uint16_t*)bsp;
// bsp += sizeof(*lenp);
// uint16_t len = *lenp;
// memcpy(d, bsp, len);
// bsp += len;
// d[len] = 0;
// if (rid == 0xFFFFFFFFFFFFFFFFULL)
// {
// strcpy(d, CPNULLSTRMARK.c_str());
// }
//#ifdef FIFO_SINK
// if (fOid < 3000)
//#endif
// if (fifo)
// {
// rw.et[rw.count++] = StringElementType(rid, d);
// if (rw.count == rw.ElementsPerGroup)
// {
// fifo->insert(rw);
// rw.count = 0;
// }
// }
// else
// {
// dlp->insert(StringElementType(rid, d));
// }
//
//#ifdef DEBUG
// cout << " -- inserting <" << rid << ", " << d << ">" << endl;
//#endif
// ridResults++;
//
// }
// }
//
// junk:
//
// if (fifo && rw.count > 0)
// fifo->insert(rw);
//
// //@bug 699: Reset StepMsgQueue
// fDec->removeQueue(uniqueID);
//
// if (fOid>=3000) dlTimes.setEndOfInputTime();
// dlp->endOfInput();
//
// if (fTableOid >= 3000)
// {
// //...Construct timestamp using ctime_r() instead of ctime() not
// //...necessarily due to re-entrancy, but because we want to strip
// //...the newline ('\n') off the end of the formatted string.
// time_t t = time(0);
// char timeString[50];
// ctime_r(&t, timeString);
// timeString[strlen(timeString)-1 ] = '\0';
//
// FifoDataList* pFifo = 0;
// uint64_t totalBlockedReadCount = 0;
// uint64_t totalBlockedWriteCount = 0;
//
// //...Sum up the blocked FIFO reads for all input associations
// size_t inDlCnt = fInputJobStepAssociation.outSize();
// for (size_t iDataList=0; iDataList<inDlCnt; iDataList++)
// {
// pFifo = fInputJobStepAssociation.outAt(iDataList)->fifoDL();
// if (pFifo)
// {
// totalBlockedReadCount += pFifo->blockedReadCount();
// }
// }
//
// //...Sum up the blocked FIFO writes for all output associations
// size_t outDlCnt = fOutputJobStepAssociation.outSize();
// for (size_t iDataList=0; iDataList<outDlCnt; iDataList++)
// {
// pFifo = fOutputJobStepAssociation.outAt(iDataList)->fifoDL();
// if (pFifo)
// {
// totalBlockedWriteCount += pFifo->blockedWriteCount();
// }
// }
//
//
//
// //...Roundoff msg byte counts to nearest KB for display
// uint64_t msgBytesInKB = fMsgBytesIn >> 10;
// uint64_t msgBytesOutKB = fMsgBytesOut >> 10;
// if (fMsgBytesIn & 512)
// msgBytesInKB++;
// if (fMsgBytesOut & 512)
// msgBytesOutKB++;
//
// // @bug 807
// if (fifo)
// fifo->totalSize(ridResults);
//
// if (traceOn())
// {
// //...Print job step completion information
// ostringstream logStr;
// logStr << "ses:" << fSessionId << " st: " << fStepId <<
// " finished at " <<
// timeString << "; PhyI/O-" << fPhysicalIO << "; CacheI/O-" <<
// fCacheIO << "; MsgsRcvd-" << msgsRecvd <<
// "; BlockedFifoIn/Out-" << totalBlockedReadCount <<
// "/" << totalBlockedWriteCount <<
// "; output size-" << ridResults << endl <<
// "\tMsgBytesIn-" << msgBytesInKB << "KB" <<
// "; MsgBytesOut-" << msgBytesOutKB << "KB" << endl <<
// "\t1st read " << dlTimes.FirstReadTimeString() <<
// "; EOI " << dlTimes.EndOfInputTimeString() << "; runtime-" <<
// JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(),dlTimes.FirstReadTime()) <<
// "s" << endl;
//
// logEnd(logStr.str().c_str());
//
// syslogReadBlockCounts(16, // exemgr subsystem
// fPhysicalIO, // # blocks read from disk
// fCacheIO, // # blocks read from cache
// 0); // # casual partition block hits
// syslogProcessingTimes(16, // exemgr subsystem
// dlTimes.FirstReadTime(), // first datalist read
// dlTimes.LastReadTime(), // last datalist read
// dlTimes.FirstInsertTime(), // first datalist write
// dlTimes.EndOfInputTime()); // last (endOfInput) datalist write
// syslogEndStep(16, // exemgr subsystem
// totalBlockedReadCount, // blocked datalist input
// totalBlockedWriteCount, // blocked datalist output
// fMsgBytesIn, // incoming msg byte count
// fMsgBytesOut); // outgoing msg byte count
// }
// }
//
}
const string pDictionaryStep::toString() const const string pDictionaryStep::toString() const
{ {
ostringstream oss; ostringstream oss;
@ -546,9 +177,6 @@ void pDictionaryStep::appendFilter(const messageqcpp::ByteStream& filter, unsign
addFilter(COP, value); addFilter(COP, value);
bs.advance(size); bs.advance(size);
} }
// fFilterString += filter;
// fFilterCount += count;
} }
void pDictionaryStep::addFilter(const Filter* f) void pDictionaryStep::addFilter(const Filter* f)

View File

@ -138,10 +138,8 @@ pDictionaryScan::pDictionaryScan(CalpontSystemCatalog::OID o, CalpontSystemCatal
, fColType(ct) , fColType(ct)
, pThread(0) , pThread(0)
, cThread(0) , cThread(0)
, fScanLbidReqLimit(jobInfo.rm->getJlScanLbidReqLimit())
, fScanLbidReqThreshold(jobInfo.rm->getJlScanLbidReqThreshold()) , fScanLbidReqThreshold(jobInfo.rm->getJlScanLbidReqThreshold())
, fStopSending(false) , fStopSending(false)
, fSingleThread(false)
, fPhysicalIO(0) , fPhysicalIO(0)
, fCacheIO(0) , fCacheIO(0)
, fMsgBytesIn(0) , fMsgBytesIn(0)
@ -915,4 +913,19 @@ void pDictionaryScan::abort()
fDec->shutdownQueue(uniqueID); fDec->shutdownQueue(uniqueID);
} }
// Unfortuneately we have 32 bits in the execplan flags, but only 16 that can be sent to
// PrimProc, so we have to convert them (throwing some away).
uint16_t pDictionaryScan::planFlagsToPrimFlags(uint32_t planFlags)
{
uint16_t flags = 0;
if (planFlags & CalpontSelectExecutionPlan::TRACE_LBIDS)
flags |= PF_LBID_TRACE;
if (planFlags & CalpontSelectExecutionPlan::PM_PROFILE)
flags |= PF_PM_PROF;
return flags;
}
} // namespace joblist } // namespace joblist

View File

@ -1,74 +0,0 @@
/* Copyright (C) 2014 InfiniDB, Inc.
This program is free software; you can redistribute it and/or
modify it under the terms of the GNU General Public License
as published by the Free Software Foundation; version 2 of
the License.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU General Public License for more details.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston,
MA 02110-1301, USA. */
/*
* $Id: primitivemsg.cpp 9210 2013-01-21 14:10:42Z rdempsey $
*/
#include <stdexcept>
using namespace std;
#include "calpontselectexecutionplan.h"
using namespace execplan;
#include "primitivemsg.h"
#include "primitivestep.h"
using namespace joblist;
namespace joblist
{
void PrimitiveMsg::send()
{
throw logic_error("somehow ended up in PrimitiveMsg::send()!");
}
void PrimitiveMsg::buildPrimitiveMessage(ISMPACKETCOMMAND, void*, void*)
{
throw logic_error("somehow ended up in PrimitiveMsg::buildPrimitiveMessage()!");
}
void PrimitiveMsg::receive()
{
throw logic_error("somehow ended up in PrimitiveMsg::receive()!");
}
void PrimitiveMsg::sendPrimitiveMessages()
{
throw logic_error("somehow ended up in PrimitiveMsg::sendPrimitiveMessages()!");
}
void PrimitiveMsg::receivePrimitiveMessages()
{
throw logic_error("somehow ended up in PrimitiveMsg::receivePrimitiveMessages()!");
}
// Unfortuneately we have 32 bits in the execplan flags, but only 16 that can be sent to
// PrimProc, so we have to convert them (throwing some away).
uint16_t PrimitiveMsg::planFlagsToPrimFlags(uint32_t planFlags)
{
uint16_t flags = 0;
if (planFlags & CalpontSelectExecutionPlan::TRACE_LBIDS)
flags |= PF_LBID_TRACE;
if (planFlags & CalpontSelectExecutionPlan::PM_PROFILE)
flags |= PF_PM_PROF;
return flags;
}
} // namespace joblist

View File

@ -93,39 +93,9 @@ enum PrimitiveStepType
AGGRFILTERSTEP AGGRFILTERSTEP
}; };
/** @brief class PrimitiveMsg
*
*/
class PrimitiveMsg
{
public:
/** @brief virtual void Send method
*/
virtual void send();
/** @brief virtual void Receive method
*/
virtual void receive();
/** @brief virtual void BuildPrimitiveMessage method
*/
virtual void buildPrimitiveMessage(ISMPACKETCOMMAND cmd, void* filterValues, void* ridArray);
virtual void sendPrimitiveMessages();
virtual void receivePrimitiveMessages();
PrimitiveMsg()
{
}
virtual ~PrimitiveMsg()
{
}
uint16_t planFlagsToPrimFlags(uint32_t planFlags);
private:
};
class pColScanStep; class pColScanStep;
class pColStep : public JobStep, public PrimitiveMsg class pColStep : public JobStep
{ {
typedef std::pair<int64_t, int64_t> element_t; typedef std::pair<int64_t, int64_t> element_t;
@ -141,48 +111,30 @@ class pColStep : public JobStep, public PrimitiveMsg
pColStep(const PassThruStep& rhs); pColStep(const PassThruStep& rhs);
virtual ~pColStep(); virtual ~pColStep(){};
/** @brief Starts processing. Set at least the RID list before calling. /** @brief Starts processing. Set at least the RID list before calling.
* *
* Starts processing. Set at least the RID list before calling this. * Starts processing. Set at least the RID list before calling this.
*/ */
virtual void run(); virtual void run(){};
/** @brief Sync's the caller with the end of execution. /** @brief Sync's the caller with the end of execution.
* *
* Does nothing. Returns when this instance is finished. * Does nothing. Returns when this instance is finished.
*/ */
virtual void join(); virtual void join(){};
virtual const std::string toString() const; virtual const std::string toString() const;
virtual bool isDictCol() const virtual bool isDictCol() const
{ {
return fIsDict; return fIsDict;
}; }
bool isExeMgr() const bool isExeMgr() const
{ {
return isEM; return isEM;
} }
/** @brief Set config parameters for this JobStep.
*
* Set the config parameters this JobStep.
*/
void initializeConfigParms();
/** @brief The main loop for the send-side thread
*
* The main loop for the primitive-issuing thread. Don't call it directly.
*/
void sendPrimitiveMessages();
/** @brief The main loop for the recv-side thread
*
* The main loop for the receive-side thread. Don't call it directly.
*/
void receivePrimitiveMessages();
/** @brief Add a filter. Use this interface when the column stores anything but 4-byte floats. /** @brief Add a filter. Use this interface when the column stores anything but 4-byte floats.
* *
* Add a filter. Use this interface when the column stores anything but 4-byte floats. * Add a filter. Use this interface when the column stores anything but 4-byte floats.
@ -197,35 +149,38 @@ class pColStep : public JobStep, public PrimitiveMsg
* this class from pColScan. Use pColScan if the every RID should be considered; it's * this class from pColScan. Use pColScan if the every RID should be considered; it's
* faster at that. * faster at that.
*/ */
void setRidList(DataList<ElementType>* rids); void setRidList(DataList<ElementType>* rids)
{
ridList = rids;
}
/** @brief Sets the String DataList to get RID values from. /** @brief Sets the String DataList to get RID values from.
* *
* Sets the string DataList to get RID values from. Filtering by RID distinguishes * Sets the string DataList to get RID values from. Filtering by RID distinguishes
* this class from pColScan. Use pColScan if the every RID should be considered; it's * this class from pColScan. Use pColScan if the every RID should be considered; it's
* faster at that. * faster at that.
*/ */
void setStrRidList(DataList<StringElementType>* strDl); void setStrRidList(DataList<StringElementType>* strDl)
{
strRidList = strDl;
}
/** @brief Set the binary operator for the filter predicate (BOP_AND or BOP_OR). /** @brief Set the binary operator for the filter predicate (BOP_AND or BOP_OR).
* *
* Set the binary operator for the filter predicate (BOP_AND or BOP_OR). * Set the binary operator for the filter predicate (BOP_AND or BOP_OR).
*/ */
void setBOP(int8_t BOP); void setBOP(int8_t BOP)
{
/** @brief Set the output type. fBOP = BOP;
* }
* Set the output type (1 = RID, 2 = Token, 3 = Both).
*/
void setOutputType(int8_t OutputType);
/** @brief Set the swallowRows flag. /** @brief Set the swallowRows flag.
* *
* *
* If true, no rows will be inserted to the output datalists. * If true, no rows will be inserted to the output datalists.
*/ */
void setSwallowRows(const bool swallowRows); void setSwallowRows(const bool swallowRows)
{
fSwallowRows = swallowRows;
}
/** @brief Get the swallowRows flag. /** @brief Get the swallowRows flag.
* *
* *
@ -263,10 +218,6 @@ class pColStep : public JobStep, public PrimitiveMsg
return fColType; return fColType;
} }
void appendFilter(const messageqcpp::ByteStream& filter, unsigned count); void appendFilter(const messageqcpp::ByteStream& filter, unsigned count);
uint32_t flushInterval() const
{
return fFlushInterval;
}
bool getFeederFlag() const bool getFeederFlag() const
{ {
return isFilterFeeder; return isFilterFeeder;
@ -276,14 +227,6 @@ class pColStep : public JobStep, public PrimitiveMsg
{ {
isFilterFeeder = filterFeeder; isFilterFeeder = filterFeeder;
} }
virtual uint64_t phyIOCount() const
{
return fPhysicalIO;
}
virtual uint64_t cacheIOCount() const
{
return fCacheIO;
}
virtual uint64_t msgsRcvdCount() const virtual uint64_t msgsRcvdCount() const
{ {
return msgsRecvd; return msgsRecvd;
@ -329,14 +272,6 @@ class pColStep : public JobStep, public PrimitiveMsg
*/ */
explicit pColStep(); explicit pColStep();
/** @brief StartPrimitiveThread
* Utility function to start worker thread that sends primitive messages
*/
void startPrimitiveThread();
/** @brief StartAggregationThread
* Utility function to start worker thread that receives result aggregation from primitive servers
*/
void startAggregationThread();
uint64_t getLBID(uint64_t rid, bool& scan); uint64_t getLBID(uint64_t rid, bool& scan);
uint64_t getFBO(uint64_t lbid); uint64_t getFBO(uint64_t lbid);
@ -347,7 +282,6 @@ class pColStep : public JobStep, public PrimitiveMsg
execplan::CalpontSystemCatalog::ColType fColType; execplan::CalpontSystemCatalog::ColType fColType;
uint32_t fFilterCount; uint32_t fFilterCount;
int8_t fBOP; int8_t fBOP;
int8_t fOutputType;
uint16_t realWidth; uint16_t realWidth;
DataList_t* ridList; DataList_t* ridList;
StrDataList* strRidList; StrDataList* strRidList;
@ -359,29 +293,18 @@ class pColStep : public JobStep, public PrimitiveMsg
bool finishedSending, recvWaiting, fIsDict; bool finishedSending, recvWaiting, fIsDict;
bool isEM; bool isEM;
int64_t ridCount; int64_t ridCount;
uint32_t fFlushInterval;
// @bug 663 - Added fSwallowRows for calpont.caltrace(16) which is TRACE_FLAGS::TRACE_NO_ROWS4. // @bug 663 - Added fSwallowRows for calpont.caltrace(16) which is TRACE_FLAGS::TRACE_NO_ROWS4.
// Running with this one will swallow rows at projection. // Running with this one will swallow rows at projection.
bool fSwallowRows; bool fSwallowRows;
uint32_t fProjectBlockReqLimit; // max number of rids to send in a scan
// request to primproc
uint32_t fProjectBlockReqThreshold; // min level of rids backlog before
// consumer will tell producer to send
// more rids scan requests to primproc
volatile bool fStopSending;
bool isFilterFeeder; bool isFilterFeeder;
uint64_t fPhysicalIO; // total physical I/O count
uint64_t fCacheIO; // total cache I/O count
uint64_t fNumBlksSkipped; // total number of block scans skipped due to CP uint64_t fNumBlksSkipped; // total number of block scans skipped due to CP
uint64_t fMsgBytesIn; // total byte count for incoming messages uint64_t fMsgBytesIn; // total byte count for incoming messages
uint64_t fMsgBytesOut; // total byte count for outcoming messages uint64_t fMsgBytesOut; // total byte count for outcoming messages
BRM::DBRM dbrm; BRM::DBRM dbrm;
// boost::shared_ptr<boost::thread> cThread; //consumer thread
// boost::shared_ptr<boost::thread> pThread; //producer thread
boost::mutex mutex; boost::mutex mutex;
boost::condition condvar; boost::condition condvar;
boost::condition flushed; boost::condition flushed;
@ -413,7 +336,7 @@ class pColStep : public JobStep, public PrimitiveMsg
* c) send messages to the primitive server as quickly as possible * c) send messages to the primitive server as quickly as possible
*/ */
class pColScanStep : public JobStep, public PrimitiveMsg class pColScanStep : public JobStep
{ {
public: public:
/** @brief pColScanStep constructor /** @brief pColScanStep constructor
@ -422,38 +345,25 @@ class pColScanStep : public JobStep, public PrimitiveMsg
const execplan::CalpontSystemCatalog::ColType& ct, const JobInfo& jobInfo); const execplan::CalpontSystemCatalog::ColType& ct, const JobInfo& jobInfo);
pColScanStep(const pColStep& rhs); pColScanStep(const pColStep& rhs);
~pColScanStep(); ~pColScanStep(){}
/** @brief Starts processing. /** @brief Starts processing.
* *
* Starts processing. * Starts processing.
*/ */
virtual void run(); virtual void run(){}
/** @brief Sync's the caller with the end of execution. /** @brief Sync's the caller with the end of execution.
* *
* Does nothing. Returns when this instance is finished. * Does nothing. Returns when this instance is finished.
*/ */
virtual void join(); virtual void join(){}
virtual bool isDictCol() const virtual bool isDictCol() const
{ {
return fIsDict; return fIsDict;
}; };
/** @brief The main loop for the send-side thread
*
* The main loop for the primitive-issuing thread. Don't call it directly.
*/
void sendPrimitiveMessages();
/** @brief The main loop for the recv-side thread
*
* The main loop for the receive-side thread. Don't call it directly.
*/
using PrimitiveMsg::receivePrimitiveMessages;
void receivePrimitiveMessages(uint64_t i = 0);
/** @brief Add a filter when the column is a 4-byte float type /** @brief Add a filter when the column is a 4-byte float type
* *
* Add a filter when the column is a 4-byte float type * Add a filter when the column is a 4-byte float type
@ -472,7 +382,11 @@ class pColScanStep : public JobStep, public PrimitiveMsg
* Set the binary operator for the filter predicates (BOP_AND or BOP_OR). * Set the binary operator for the filter predicates (BOP_AND or BOP_OR).
* It is initialized to OR. * It is initialized to OR.
*/ */
void setBOP(int8_t BOP); // AND or OR void setBOP(int8_t BOP) // AND or OR
{
fBOP = BOP;
}
int8_t BOP() const int8_t BOP() const
{ {
return fBOP; return fBOP;
@ -496,17 +410,6 @@ class pColScanStep : public JobStep, public PrimitiveMsg
return fFilterString; return fFilterString;
} }
void setSingleThread(bool b);
bool getSingleThread()
{
return fSingleThread;
}
/** @brief Set the output type.
*
* Set the output type (1 = RID, 2 = Token, 3 = Both).pColScan
*/
void setOutputType(int8_t OutputType);
uint32_t filterCount() const uint32_t filterCount() const
{ {
return fFilterCount; return fFilterCount;
@ -532,18 +435,6 @@ class pColScanStep : public JobStep, public PrimitiveMsg
return fRm; return fRm;
} }
virtual uint64_t phyIOCount() const
{
return fPhysicalIO;
}
virtual uint64_t cacheIOCount() const
{
return fCacheIO;
}
virtual uint64_t msgsRcvdCount() const
{
return recvCount;
}
virtual uint64_t msgBytesIn() const virtual uint64_t msgBytesIn() const
{ {
return fMsgBytesIn; return fMsgBytesIn;
@ -599,18 +490,12 @@ class pColScanStep : public JobStep, public PrimitiveMsg
// pColScanStep& operator=(const pColScanStep& rhs); // pColScanStep& operator=(const pColScanStep& rhs);
typedef boost::shared_ptr<boost::thread> SPTHD; typedef boost::shared_ptr<boost::thread> SPTHD;
void startPrimitiveThread();
void startAggregationThread();
void initializeConfigParms();
void sendAPrimitiveMessage(ISMPacketHeader& ism, BRM::LBID_t msgLbidStart, uint32_t msgLbidCount);
uint64_t getFBO(uint64_t lbid); uint64_t getFBO(uint64_t lbid);
bool isEmptyVal(const uint8_t* val8) const; bool isEmptyVal(const uint8_t* val8) const;
ResourceManager* fRm; ResourceManager* fRm;
ColByScanRangeRequestHeader fMsgHeader; ColByScanRangeRequestHeader fMsgHeader;
SPTHD fConsumerThread; SPTHD fConsumerThread;
/// number of threads on the receive side
uint32_t fNumThreads;
SPTHD* fProducerThread; SPTHD* fProducerThread;
messageqcpp::ByteStream fFilterString; messageqcpp::ByteStream fFilterString;
@ -619,16 +504,10 @@ class pColScanStep : public JobStep, public PrimitiveMsg
execplan::CalpontSystemCatalog::OID fTableOid; execplan::CalpontSystemCatalog::OID fTableOid;
execplan::CalpontSystemCatalog::ColType fColType; execplan::CalpontSystemCatalog::ColType fColType;
int8_t fBOP; int8_t fBOP;
int8_t fOutputType;
uint32_t sentCount;
uint32_t recvCount;
BRM::LBIDRange_v lbidRanges; BRM::LBIDRange_v lbidRanges;
BRM::DBRM dbrm; BRM::DBRM dbrm;
SP_LBIDList lbidList; SP_LBIDList lbidList;
boost::mutex mutex;
boost::mutex dlMutex;
boost::mutex cpMutex;
boost::condition condvar; boost::condition condvar;
boost::condition condvarWakeupProducer; boost::condition condvarWakeupProducer;
bool finishedSending, sendWaiting, rDoNothing, fIsDict; bool finishedSending, sendWaiting, rDoNothing, fIsDict;
@ -638,17 +517,7 @@ class pColScanStep : public JobStep, public PrimitiveMsg
uint32_t extentSize, divShift, ridsPerBlock, rpbShift, numExtents; uint32_t extentSize, divShift, ridsPerBlock, rpbShift, numExtents;
// config::Config *fConfig; // config::Config *fConfig;
uint32_t fScanLbidReqLimit; // max number of LBIDs to send in a scan
// request to primproc
uint32_t fScanLbidReqThreshold; // min level of scan LBID backlog before
// consumer will tell producer to send
// more LBID scan requests to primproc
bool fStopSending;
bool fSingleThread;
bool isFilterFeeder; bool isFilterFeeder;
uint64_t fPhysicalIO; // total physical I/O count
uint64_t fCacheIO; // total cache I/O count
uint64_t fNumBlksSkipped; // total number of block scans skipped due to CP uint64_t fNumBlksSkipped; // total number of block scans skipped due to CP
uint64_t fMsgBytesIn; // total byte count for incoming messages uint64_t fMsgBytesIn; // total byte count for incoming messages
uint64_t fMsgBytesOut; // total byte count for outcoming messages uint64_t fMsgBytesOut; // total byte count for outcoming messages
@ -667,35 +536,10 @@ class pColScanStep : public JobStep, public PrimitiveMsg
friend class TupleBPS; friend class TupleBPS;
}; };
#if 0
class pIdxStep : public JobStep
{
public:
/** @brief pIdxStep constructor
* @param in the inputAssociation pointer
* @param out the outputAssociation pointer
* @param ec the DistributedEngineComm pointer
*/
pIdxStep(JobStepAssociation* in, JobStepAssociation* out, DistributedEngineComm* ec);
/** @brief virtual void Run method
*/
virtual void run();
private:
pIdxStep();
void startPrimitveThread();
void startAggregationThread();
protected:
DistributedEngineComm* fDec;
JobStepAssociation* fInputJobStepAssociation;
JobStepAssociation* fOutputJobStepAssociation;
};
#endif
/** @brief class pDictionaryStep /** @brief class pDictionaryStep
* *
*/ */
class pDictionaryStep : public JobStep, public PrimitiveMsg class pDictionaryStep : public JobStep
{ {
public: public:
/** @brief pDictionaryStep constructor /** @brief pDictionaryStep constructor
@ -704,17 +548,22 @@ class pDictionaryStep : public JobStep, public PrimitiveMsg
pDictionaryStep(execplan::CalpontSystemCatalog::OID oid, execplan::CalpontSystemCatalog::OID tabelOid, pDictionaryStep(execplan::CalpontSystemCatalog::OID oid, execplan::CalpontSystemCatalog::OID tabelOid,
const execplan::CalpontSystemCatalog::ColType& ct, const JobInfo& jobInfo); const execplan::CalpontSystemCatalog::ColType& ct, const JobInfo& jobInfo);
virtual ~pDictionaryStep(); virtual ~pDictionaryStep(){}
/** @brief virtual void Run method /** @brief virtual void Run method
*/ */
virtual void run(); virtual void run(){}
virtual void join(); virtual void join(){}
// void setOutList(StringDataList* rids); // void setOutList(StringDataList* rids);
void setInputList(DataList_t* rids); void setInputList(DataList_t* rids)
void setBOP(int8_t b); {
void sendPrimitiveMessages(); requestList = rids;
void receivePrimitiveMessages(); }
void setBOP(int8_t b)
{
fBOP = b;
}
virtual const std::string toString() const; virtual const std::string toString() const;
@ -735,14 +584,6 @@ class pDictionaryStep : public JobStep, public PrimitiveMsg
{ {
return fTableOid; return fTableOid;
} }
virtual uint64_t phyIOCount() const
{
return fPhysicalIO;
}
virtual uint64_t cacheIOCount() const
{
return fCacheIO;
}
virtual uint64_t msgsRcvdCount() const virtual uint64_t msgsRcvdCount() const
{ {
return msgsRecvd; return msgsRecvd;
@ -780,8 +621,6 @@ class pDictionaryStep : public JobStep, public PrimitiveMsg
private: private:
pDictionaryStep(); pDictionaryStep();
void startPrimitiveThread();
void startAggregationThread();
boost::shared_ptr<execplan::CalpontSystemCatalog> sysCat; boost::shared_ptr<execplan::CalpontSystemCatalog> sysCat;
execplan::CalpontSystemCatalog::OID fOid; execplan::CalpontSystemCatalog::OID fOid;
@ -804,8 +643,6 @@ class pDictionaryStep : public JobStep, public PrimitiveMsg
boost::mutex mutex; boost::mutex mutex;
boost::condition condvar; boost::condition condvar;
uint32_t fInterval; uint32_t fInterval;
uint64_t fPhysicalIO; // total physical I/O count
uint64_t fCacheIO; // total cache I/O count
uint64_t fMsgBytesIn; // total byte count for incoming messages uint64_t fMsgBytesIn; // total byte count for incoming messages
uint64_t fMsgBytesOut; // total byte count for outcoming messages uint64_t fMsgBytesOut; // total byte count for outcoming messages
uint32_t uniqueID; uint32_t uniqueID;
@ -828,7 +665,7 @@ class pDictionaryStep : public JobStep, public PrimitiveMsg
/** @brief class pDictionaryScan /** @brief class pDictionaryScan
* *
*/ */
class pDictionaryScan : public JobStep, public PrimitiveMsg class pDictionaryScan : public JobStep
{ {
public: public:
/** @brief pDictionaryScan constructor /** @brief pDictionaryScan constructor
@ -953,6 +790,7 @@ class pDictionaryScan : public JobStep, public PrimitiveMsg
private: private:
pDictionaryScan(); pDictionaryScan();
uint16_t planFlagsToPrimFlags(uint32_t planFlags);
void startPrimitiveThread(); void startPrimitiveThread();
void startAggregationThread(); void startAggregationThread();
void initializeConfigParms(); void initializeConfigParms();
@ -990,12 +828,10 @@ class pDictionaryScan : public JobStep, public PrimitiveMsg
uint64_t extentSize; uint64_t extentSize;
uint64_t divShift; uint64_t divShift;
uint64_t numExtents; uint64_t numExtents;
uint32_t fScanLbidReqLimit; // max number of LBIDs to send in a scan
// request to primproc // request to primproc
uint32_t fScanLbidReqThreshold; // min level of scan LBID backlog before uint32_t fScanLbidReqThreshold; // min level of scan LBID backlog before
// consumer will tell producer to send // consumer will tell producer to send
bool fStopSending; bool fStopSending;
bool fSingleThread;
uint64_t fPhysicalIO; // total physical I/O count uint64_t fPhysicalIO; // total physical I/O count
uint64_t fCacheIO; // total cache I/O count uint64_t fCacheIO; // total cache I/O count
uint64_t fMsgBytesIn; // total byte count for incoming messages uint64_t fMsgBytesIn; // total byte count for incoming messages
@ -1018,7 +854,7 @@ class pDictionaryScan : public JobStep, public PrimitiveMsg
void destroyEqualityFilter(); void destroyEqualityFilter();
}; };
class BatchPrimitive : public JobStep, public PrimitiveMsg, public DECEventListener class BatchPrimitive : public JobStep, public DECEventListener
{ {
public: public:
BatchPrimitive(const JobInfo& jobInfo) : JobStep(jobInfo) BatchPrimitive(const JobInfo& jobInfo) : JobStep(jobInfo)
@ -1697,7 +1533,7 @@ class FilterStep : public JobStep
/** @brief class PassThruStep /** @brief class PassThruStep
* *
*/ */
class PassThruStep : public JobStep, public PrimitiveMsg class PassThruStep : public JobStep
{ {
typedef std::pair<int64_t, int64_t> element_t; typedef std::pair<int64_t, int64_t> element_t;

View File

@ -64,7 +64,6 @@ ResourceManager::ResourceManager(bool runningInExeMgr, config::Config* aConfig)
, fHjNumThreads(defaultNumThreads) , fHjNumThreads(defaultNumThreads)
, fJlProcessorThreadsPerScan(defaultProcessorThreadsPerScan) , fJlProcessorThreadsPerScan(defaultProcessorThreadsPerScan)
, fJlNumScanReceiveThreads(defaultScanReceiveThreads) , fJlNumScanReceiveThreads(defaultScanReceiveThreads)
, fTwNumThreads(defaultNumThreads)
, fJlMaxOutstandingRequests(defaultMaxOutstandingRequests) , fJlMaxOutstandingRequests(defaultMaxOutstandingRequests)
, fHJUmMaxMemorySmallSideDistributor( , fHJUmMaxMemorySmallSideDistributor(
fHashJoinStr, "UmMaxMemorySmallSide", fHashJoinStr, "UmMaxMemorySmallSide",
@ -101,7 +100,6 @@ ResourceManager::ResourceManager(bool runningInExeMgr, config::Config* aConfig)
{ {
fHjNumThreads = fNumCores; fHjNumThreads = fNumCores;
fJlNumScanReceiveThreads = fNumCores; fJlNumScanReceiveThreads = fNumCores;
fTwNumThreads = fNumCores;
} }
// possibly override any calculated values // possibly override any calculated values
@ -139,11 +137,6 @@ ResourceManager::ResourceManager(bool runningInExeMgr, config::Config* aConfig)
fDECConnectionsPerQuery = fDECConnectionsPerQuery =
(fDECConnectionsPerQuery) ? fDECConnectionsPerQuery : getPsConnectionsPerPrimProc(); (fDECConnectionsPerQuery) ? fDECConnectionsPerQuery : getPsConnectionsPerPrimProc();
temp = getIntVal(fTupleWSDLStr, "NumThreads", -1);
if (temp > 0)
fTwNumThreads = temp;
pmJoinMemLimit = getUintVal(fHashJoinStr, "PmMaxMemorySmallSide", defaultHJPmMaxMemorySmallSide); pmJoinMemLimit = getUintVal(fHashJoinStr, "PmMaxMemorySmallSide", defaultHJPmMaxMemorySmallSide);
// Need to use different limits if this instance isn't running on the UM, // Need to use different limits if this instance isn't running on the UM,
@ -317,106 +310,6 @@ void ResourceManager::logResourceChangeMessage(logging::LOG_TYPE logType, uint32
log.logMessage(logType, mid, args, logging::LoggingID(5, sessionID)); log.logMessage(logType, mid, args, logging::LoggingID(5, sessionID));
} }
void ResourceManager::emServerThreads()
{
}
void ResourceManager::emServerQueueSize()
{
}
void ResourceManager::emSecondsBetweenMemChecks()
{
}
void ResourceManager::emMaxPct()
{
}
void ResourceManager::emPriority()
{
}
void ResourceManager::emExecQueueSize()
{
}
void ResourceManager::hjNumThreads()
{
}
void ResourceManager::hjMaxBuckets()
{
}
void ResourceManager::hjMaxElems()
{
}
void ResourceManager::hjFifoSizeLargeSide()
{
}
void ResourceManager::hjPmMaxMemorySmallSide()
{
}
void ResourceManager::jlFlushInterval()
{
}
void ResourceManager::jlFifoSize()
{
}
void ResourceManager::jlScanLbidReqLimit()
{
}
void ResourceManager::jlScanLbidReqThreshold()
{
}
void ResourceManager::jlProjectBlockReqLimit()
{
}
void ResourceManager::jlProjectBlockReqThreshold()
{
}
void ResourceManager::jlNumScanReceiveThreads()
{
}
void ResourceManager::psCount()
{
}
void ResourceManager::psConnectionsPerPrimProc()
{
}
void ResourceManager::psLBID_Shift()
{
}
void ResourceManager::scTempDiskPath()
{
}
void ResourceManager::scTempSaveSize()
{
}
void ResourceManager::scWorkingDir()
{
}
void ResourceManager::twMaxSize()
{
}
void ResourceManager::twInitialCapacity()
{
}
void ResourceManager::twMaxBuckets()
{
}
void ResourceManager::twNumThreads()
{
}
void ResourceManager::zdl_MaxElementsInMem()
{
}
void ResourceManager::zdl_MaxElementsPerBucket()
{
}
void ResourceManager::hbrPredicate()
{
}
bool ResourceManager::getMysqldInfo(std::string& h, std::string& u, std::string& w, unsigned int& p) const bool ResourceManager::getMysqldInfo(std::string& h, std::string& u, std::string& w, unsigned int& p) const
{ {
static const std::string hostUserUnassignedValue("unassigned"); static const std::string hostUserUnassignedValue("unassigned");

View File

@ -52,31 +52,18 @@ const uint32_t defaultNumThreads = 8;
// joblistfactory // joblistfactory
const uint32_t defaultFlushInterval = 8 * 1024; const uint32_t defaultFlushInterval = 8 * 1024;
const uint32_t defaultFifoSize = 10; const uint32_t defaultFifoSize = 10;
const uint32_t defaultHJFifoSizeLargeSide = 128;
const uint64_t defaultHJMaxElems = 512 * 1024; // hashjoin uses 8192 const uint64_t defaultHJMaxElems = 512 * 1024; // hashjoin uses 8192
const int defaultHJMaxBuckets = 32; // hashjoin uses 4 const int defaultHJMaxBuckets = 32; // hashjoin uses 4
const uint64_t defaultHJPmMaxMemorySmallSide = 1 * 1024 * 1024 * 1024ULL; const uint64_t defaultHJPmMaxMemorySmallSide = 1 * 1024 * 1024 * 1024ULL;
const uint64_t defaultHJUmMaxMemorySmallSide = 4 * 1024 * 1024 * 1024ULL; const uint64_t defaultHJUmMaxMemorySmallSide = 4 * 1024 * 1024 * 1024ULL;
const uint32_t defaultTempSaveSize = defaultHJMaxElems;
const uint64_t defaultTotalUmMemory = 8 * 1024 * 1024 * 1024ULL; const uint64_t defaultTotalUmMemory = 8 * 1024 * 1024 * 1024ULL;
const uint64_t defaultHUATotalMem = 8 * 1024 * 1024 * 1024ULL;
const uint32_t defaultTupleDLMaxSize = 64 * 1024;
const uint32_t defaultJLThreadPoolSize = 100; const uint32_t defaultJLThreadPoolSize = 100;
// pcolscan.cpp // pcolscan.cpp
const uint32_t defaultScanLbidReqLimit = 10000;
const uint32_t defaultScanLbidReqThreshold = 5000; const uint32_t defaultScanLbidReqThreshold = 5000;
const uint32_t defaultLogicalBlocksPerScan = 1024; // added for bug 1264. const uint32_t defaultLogicalBlocksPerScan = 1024; // added for bug 1264.
const uint32_t defaultScanBlockThreshhold = 10000; // in jobstep.h
const uint32_t defaultScanReceiveThreads = 8; const uint32_t defaultScanReceiveThreads = 8;
// pcolstep.cpp
const uint32_t defaultProjectBlockReqLimit = 32 * 1024;
const uint32_t defaultProjectBlockReqThreshold = 16 * 1024; // 256 in jobstep.h
// BatchPrimitiveStep // BatchPrimitiveStep
const uint32_t defaultRequestSize = 1; const uint32_t defaultRequestSize = 1;
const uint32_t defaultMaxOutstandingRequests = 20; const uint32_t defaultMaxOutstandingRequests = 20;
@ -85,15 +72,6 @@ const uint32_t defaultJoinerChunkSize = 16 * 1024 * 1024;
// bucketreuse // bucketreuse
const std::string defaultTempDiskPath = "/tmp"; const std::string defaultTempDiskPath = "/tmp";
const std::string defaultWorkingDir = "."; //"/tmp";
// largedatalist
const uint32_t defaultLDLMaxElements = 32 * 1024 * 1024;
// zdl
const uint64_t defaultMaxElementsInMem = 32 * 1024 * 1024;
const uint64_t defaultNumBuckets = 128;
const uint64_t defaultMaxElementsPerBuckert = 16 * 1024 * 1024;
const int defaultEMServerThreads = 50; const int defaultEMServerThreads = 50;
const int defaultEMSecondsBetweenMemChecks = 1; const int defaultEMSecondsBetweenMemChecks = 1;
@ -101,11 +79,8 @@ const int defaultEMMaxPct = 95;
const int defaultEMPriority = 21; // @Bug 3385 const int defaultEMPriority = 21; // @Bug 3385
const int defaultEMExecQueueSize = 20; const int defaultEMExecQueueSize = 20;
const uint64_t defaultInitialCapacity = 1024 * 1024;
const int defaultTWMaxBuckets = 256;
const int defaultPSCount = 0; const int defaultPSCount = 0;
const int defaultConnectionsPerPrimProc = 1; const int defaultConnectionsPerPrimProc = 1;
const uint32_t defaultLBID_Shift = 13;
const uint64_t defaultExtentRows = 8 * 1024 * 1024; const uint64_t defaultExtentRows = 8 * 1024 * 1024;
// DMLProc // DMLProc
@ -120,13 +95,8 @@ const uint64_t defaultRowsPerBatch = 10000;
/* HJ CP feedback, see bug #1465 */ /* HJ CP feedback, see bug #1465 */
const uint32_t defaultHjCPUniqueLimit = 100; const uint32_t defaultHjCPUniqueLimit = 100;
// Order By and Limit
const uint64_t defaultOrderByLimitMaxMemory = 1 * 1024 * 1024 * 1024ULL;
const uint64_t defaultDECThrottleThreshold = 200000000; // ~200 MB const uint64_t defaultDECThrottleThreshold = 200000000; // ~200 MB
const uint8_t defaultUseCpimport = 1;
const bool defaultAllowDiskAggregation = false; const bool defaultAllowDiskAggregation = false;
/** @brief ResourceManager /** @brief ResourceManager
@ -172,7 +142,7 @@ class ResourceManager
{ {
return getUintVal(fExeMgrStr, "MaxPct", defaultEMMaxPct); return getUintVal(fExeMgrStr, "MaxPct", defaultEMMaxPct);
} }
EXPORT int getEmPriority() const; EXPORT int getEmPriority() const; // FOr Windows only
int getEmExecQueueSize() const int getEmExecQueueSize() const
{ {
return getIntVal(fExeMgrStr, "ExecQueueSize", defaultEMExecQueueSize); return getIntVal(fExeMgrStr, "ExecQueueSize", defaultEMExecQueueSize);
@ -200,10 +170,6 @@ class ResourceManager
{ {
return getUintVal(fHashJoinStr, "MaxElems", defaultHJMaxElems); return getUintVal(fHashJoinStr, "MaxElems", defaultHJMaxElems);
} }
uint32_t getHjFifoSizeLargeSide() const
{
return getUintVal(fHashJoinStr, "FifoSizeLargeSide", defaultHJFifoSizeLargeSide);
}
uint32_t getHjCPUniqueLimit() const uint32_t getHjCPUniqueLimit() const
{ {
return getUintVal(fHashJoinStr, "CPUniqueLimit", defaultHjCPUniqueLimit); return getUintVal(fHashJoinStr, "CPUniqueLimit", defaultHjCPUniqueLimit);
@ -221,10 +187,6 @@ class ResourceManager
{ {
return getUintVal(fJobListStr, "FifoSize", defaultFifoSize); return getUintVal(fJobListStr, "FifoSize", defaultFifoSize);
} }
uint32_t getJlScanLbidReqLimit() const
{
return getUintVal(fJobListStr, "ScanLbidReqLimit", defaultScanLbidReqLimit);
}
uint32_t getJlScanLbidReqThreshold() const uint32_t getJlScanLbidReqThreshold() const
{ {
return getUintVal(fJobListStr, "ScanLbidReqThreshold", defaultScanLbidReqThreshold); return getUintVal(fJobListStr, "ScanLbidReqThreshold", defaultScanLbidReqThreshold);
@ -250,14 +212,6 @@ class ResourceManager
{ {
return getUintVal(fJobListStr, "LogicalBlocksPerScan", defaultLogicalBlocksPerScan); return getUintVal(fJobListStr, "LogicalBlocksPerScan", defaultLogicalBlocksPerScan);
} }
uint32_t getJlProjectBlockReqLimit() const
{
return getUintVal(fJobListStr, "ProjectBlockReqLimit", defaultProjectBlockReqLimit);
}
uint32_t getJlProjectBlockReqThreshold() const
{
return getUintVal(fJobListStr, "ProjectBlockReqThreshold", defaultProjectBlockReqThreshold);
}
uint32_t getJlNumScanReceiveThreads() const uint32_t getJlNumScanReceiveThreads() const
{ {
return fJlNumScanReceiveThreads; return fJlNumScanReceiveThreads;
@ -290,49 +244,15 @@ class ResourceManager
{ {
return getUintVal(fPrimitiveServersStr, "ConnectionsPerPrimProc", defaultConnectionsPerPrimProc); return getUintVal(fPrimitiveServersStr, "ConnectionsPerPrimProc", defaultConnectionsPerPrimProc);
} }
uint32_t getPsLBID_Shift() const
{
return getUintVal(fPrimitiveServersStr, "LBID_Shift", defaultLBID_Shift);
}
std::string getScTempDiskPath() const std::string getScTempDiskPath() const
{ {
return startup::StartUp::tmpDir(); return startup::StartUp::tmpDir();
} }
uint64_t getScTempSaveSize() const
{
return getUintVal(fSystemConfigStr, "TempSaveSize", defaultTempSaveSize);
}
std::string getScWorkingDir() const std::string getScWorkingDir() const
{ {
return startup::StartUp::tmpDir(); return startup::StartUp::tmpDir();
} }
uint32_t getTwMaxSize() const
{
return getUintVal(fTupleWSDLStr, "MaxSize", defaultTupleDLMaxSize);
}
uint64_t getTwInitialCapacity() const
{
return getUintVal(fTupleWSDLStr, "InitialCapacity", defaultInitialCapacity);
}
int getTwMaxBuckets() const
{
return getUintVal(fTupleWSDLStr, "MaxBuckets", defaultTWMaxBuckets);
}
uint8_t getTwNumThreads() const
{
return fTwNumThreads;
} // getUintVal(fTupleWSDLStr, "NumThreads", defaultNumThreads ); }
uint64_t getZdl_MaxElementsInMem() const
{
return getUintVal(fZDLStr, "ZDL_MaxElementsInMem", defaultMaxElementsInMem);
}
uint64_t getZdl_MaxElementsPerBucket() const
{
return getUintVal(fZDLStr, "ZDL_MaxElementsPerBucket", defaultMaxElementsPerBuckert);
}
uint64_t getExtentRows() const uint64_t getExtentRows() const
{ {
return getUintVal(fExtentMapStr, "ExtentRows", defaultExtentRows); return getUintVal(fExtentMapStr, "ExtentRows", defaultExtentRows);
@ -347,13 +267,6 @@ class ResourceManager
return getUintVal(fPrimitiveServersStr, "Count", 1); return getUintVal(fPrimitiveServersStr, "Count", 1);
} }
std::vector<std::string> getHbrPredicate() const
{
std::vector<std::string> columns;
fConfig->getConfig(fHashBucketReuseStr, "Predicate", columns);
return columns;
}
uint64_t getDMLMaxDeleteRows() const uint64_t getDMLMaxDeleteRows() const
{ {
return getUintVal(fDMLProcStr, "MaxDeleteRows", defaultDMLMaxDeleteRows); return getUintVal(fDMLProcStr, "MaxDeleteRows", defaultDMLMaxDeleteRows);
@ -364,17 +277,6 @@ class ResourceManager
return getUintVal(fBatchInsertStr, "RowsPerBatch", defaultRowsPerBatch); return getUintVal(fBatchInsertStr, "RowsPerBatch", defaultRowsPerBatch);
} }
uint8_t getUseCpimport() const
{
int val = getIntVal(fBatchInsertStr, "UseCpimport", defaultUseCpimport);
return val;
}
uint64_t getOrderByLimitMaxMemory() const
{
return getUintVal(fOrderByLimitStr, "MaxMemory", defaultOrderByLimitMaxMemory);
}
uint64_t getDECThrottleThreshold() const uint64_t getDECThrottleThreshold() const
{ {
return getUintVal(fJobListStr, "DECThrottleThreshold", defaultDECThrottleThreshold); return getUintVal(fJobListStr, "DECThrottleThreshold", defaultDECThrottleThreshold);
@ -455,32 +357,6 @@ class ResourceManager
fHJUmMaxMemorySmallSideDistributor.returnResource(mem); fHJUmMaxMemorySmallSideDistributor.returnResource(mem);
} }
EXPORT void jlFlushInterval();
EXPORT void jlFifoSize();
EXPORT void jlScanLbidReqLimit();
EXPORT void jlScanLbidReqThreshold();
EXPORT void jlProjectBlockReqLimit();
EXPORT void jlProjectBlockReqThreshold();
EXPORT void jlNumScanReceiveThreads();
EXPORT void psCount();
EXPORT void psConnectionsPerPrimProc();
EXPORT void psLBID_Shift();
EXPORT void scTempDiskPath();
EXPORT void scTempSaveSize();
EXPORT void scWorkingDir();
EXPORT void twMaxSize();
EXPORT void twInitialCapacity();
EXPORT void twMaxBuckets();
EXPORT void twNumThreads();
EXPORT void zdl_MaxElementsInMem();
EXPORT void zdl_MaxElementsPerBucket();
EXPORT void hbrPredicate();
void setTraceFlags(uint32_t flags) void setTraceFlags(uint32_t flags)
{ {
fTraceFlags = flags; fTraceFlags = flags;
@ -577,12 +453,9 @@ class ResourceManager
std::string fExeMgrStr; std::string fExeMgrStr;
inline static const std::string fHashJoinStr = "HashJoin"; inline static const std::string fHashJoinStr = "HashJoin";
inline static const std::string fHashBucketReuseStr = "HashBucketReuse";
inline static const std::string fJobListStr = "JobList"; inline static const std::string fJobListStr = "JobList";
inline static const std::string fPrimitiveServersStr = "PrimitiveServers"; inline static const std::string fPrimitiveServersStr = "PrimitiveServers";
/*static const*/ std::string fSystemConfigStr; /*static const*/ std::string fSystemConfigStr;
inline static const std::string fTupleWSDLStr = "TupleWSDL";
inline static const std::string fZDLStr = "ZDL";
inline static const std::string fExtentMapStr = "ExtentMap"; inline static const std::string fExtentMapStr = "ExtentMap";
/*static const*/ std::string fDMLProcStr; /*static const*/ std::string fDMLProcStr;
/*static const*/ std::string fBatchInsertStr; /*static const*/ std::string fBatchInsertStr;
@ -596,7 +469,6 @@ class ResourceManager
unsigned fHjNumThreads; unsigned fHjNumThreads;
uint32_t fJlProcessorThreadsPerScan; uint32_t fJlProcessorThreadsPerScan;
uint32_t fJlNumScanReceiveThreads; uint32_t fJlNumScanReceiveThreads;
uint8_t fTwNumThreads;
uint32_t fJlMaxOutstandingRequests; uint32_t fJlMaxOutstandingRequests;
/* old HJ support */ /* old HJ support */

View File

@ -9,61 +9,6 @@
<Port>8601</Port> <Port>8601</Port>
<Module>unassigned</Module> <Module>unassigned</Module>
</ExeMgr1> </ExeMgr1>
<JobProc>
<IPAddr>0.0.0.0</IPAddr>
<Port>8602</Port>
</JobProc>
<ProcMgr>
<IPAddr>127.0.0.1</IPAddr>
<Port>8603</Port>
</ProcMgr>
<ProcMgr_Alarm>
<IPAddr>127.0.0.1</IPAddr>
<Port>8606</Port>
</ProcMgr_Alarm>
<ProcStatusControl>
<IPAddr>127.0.0.1</IPAddr>
<Port>8604</Port>
</ProcStatusControl>
<ProcStatusControlStandby>
<IPAddr>0.0.0.0</IPAddr>
<Port>8605</Port>
</ProcStatusControlStandby>
<!-- Disabled
<ProcHeartbeatControl>
<IPAddr>0.0.0.0</IPAddr>
<Port>8605</Port>
</ProcHeartbeatControl>
-->
<!-- ProcessMonitor Port: 8800 - 8820 is reserved to support External Modules-->
<localhost_ProcessMonitor>
<IPAddr>127.0.0.1</IPAddr>
<Port>8800</Port>
</localhost_ProcessMonitor>
<dm1_ProcessMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8800</Port>
</dm1_ProcessMonitor>
<um1_ProcessMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8800</Port>
</um1_ProcessMonitor>
<pm1_ProcessMonitor>
<IPAddr>127.0.0.1</IPAddr>
<Port>8800</Port>
</pm1_ProcessMonitor>
<dm1_ServerMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8622</Port>
</dm1_ServerMonitor>
<um1_ServerMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8622</Port>
</um1_ServerMonitor>
<pm1_ServerMonitor>
<IPAddr>127.0.0.1</IPAddr>
<Port>8622</Port>
</pm1_ServerMonitor>
<pm1_WriteEngineServer> <pm1_WriteEngineServer>
<IPAddr>127.0.0.1</IPAddr> <IPAddr>127.0.0.1</IPAddr>
<Port>8630</Port> <Port>8630</Port>
@ -85,7 +30,6 @@
<ProcessorThreshold>128</ProcessorThreshold> <ProcessorThreshold>128</ProcessorThreshold>
<ProcessorQueueSize>10K</ProcessorQueueSize> <!-- minimum of extent size 8192 --> <ProcessorQueueSize>10K</ProcessorQueueSize> <!-- minimum of extent size 8192 -->
<DebugLevel>0</DebugLevel> <DebugLevel>0</DebugLevel>
<LBID_Shift>13</LBID_Shift>
<ColScanBufferSizeBlocks>512</ColScanBufferSizeBlocks> <ColScanBufferSizeBlocks>512</ColScanBufferSizeBlocks>
<ColScanReadAheadBlocks>512</ColScanReadAheadBlocks> <!-- s/b factor of extent size 8192 --> <ColScanReadAheadBlocks>512</ColScanReadAheadBlocks> <!-- s/b factor of extent size 8192 -->
<!-- <BPPCount>16</BPPCount> --> <!-- Default num cores * 2. A cap on the number of simultaneous primitives per jobstep --> <!-- <BPPCount>16</BPPCount> --> <!-- Default num cores * 2. A cap on the number of simultaneous primitives per jobstep -->
@ -96,206 +40,40 @@
<!-- <MediumPriorityPercentage>30</MediumPriorityPercentage> --> <!-- <MediumPriorityPercentage>30</MediumPriorityPercentage> -->
<!-- <LowPriorityPercentage>10</LowPriorityPercentage> --> <!-- <LowPriorityPercentage>10</LowPriorityPercentage> -->
<DirectIO>y</DirectIO> <DirectIO>y</DirectIO>
<HighPriorityPercentage/> <HighPriorityPercentage/>
<MediumPriorityPercentage/> <MediumPriorityPercentage/>
<LowPriorityPercentage/> <LowPriorityPercentage/>
</PrimitiveServers> </PrimitiveServers>
<PMS1> <PMS1>
<IPAddr>127.0.0.1</IPAddr> <IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port> <Port>8620</Port>
</PMS1> </PMS1>
<PMS2>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS2>
<PMS3>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS3>
<PMS4>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS4>
<PMS5>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS5>
<PMS6>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS6>
<PMS7>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS7>
<PMS8>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS8>
<PMS9>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS9>
<PMS10>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS10>
<PMS11>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS11>
<PMS12>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS12>
<PMS13>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS13>
<PMS14>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS14>
<PMS15>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS15>
<PMS16>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS16>
<PMS17>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS17>
<PMS18>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS18>
<PMS19>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS19>
<PMS20>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS20>
<PMS21>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS21>
<PMS22>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS22>
<PMS23>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS23>
<PMS24>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS24>
<PMS25>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS25>
<PMS26>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS26>
<PMS27>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS27>
<PMS28>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS28>
<PMS29>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS29>
<PMS30>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS30>
<PMS31>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS31>
<PMS32>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS32>
<SystemConfig> <SystemConfig>
<SystemName>columnstore-1</SystemName> <SystemName>columnstore-1</SystemName>
<ParentOAMModuleName>pm1</ParentOAMModuleName> <ParentOAMModuleName>pm1</ParentOAMModuleName>
<StandbyOAMModuleName>unassigned</StandbyOAMModuleName>
<PrimaryUMModuleName>pm1</PrimaryUMModuleName> <PrimaryUMModuleName>pm1</PrimaryUMModuleName>
<ModuleHeartbeatPeriod>1</ModuleHeartbeatPeriod>
<ModuleHeartbeatCount>3</ModuleHeartbeatCount>
<ModuleProcMonWaitCount>12</ModuleProcMonWaitCount> // 2.5 minutes
<!-- Disabled
<ProcessHeartbeatPeriod>-1</ProcessHeartbeatPeriod>
-->
<!-- Warning: Do not change this value once database is built --> <!-- Warning: Do not change this value once database is built -->
<DBRootCount>1</DBRootCount> <DBRootCount>1</DBRootCount>
<DBRoot1>/var/lib/columnstore/data1</DBRoot1> <DBRoot1>/var/lib/columnstore/data1</DBRoot1>
<DBRMRoot>/var/lib/columnstore/data1/systemFiles/dbrm/BRM_saves</DBRMRoot> <DBRMRoot>/var/lib/columnstore/data1/systemFiles/dbrm/BRM_saves</DBRMRoot>
<TableLockSaveFile>/var/lib/columnstore/data1/systemFiles/dbrm/tablelocks</TableLockSaveFile> <TableLockSaveFile>/var/lib/columnstore/data1/systemFiles/dbrm/tablelocks</TableLockSaveFile>
<DBRMTimeOut>15</DBRMTimeOut> <!-- in seconds --> <DBRMTimeOut>15</DBRMTimeOut> <!-- in seconds -->
<DBRMSnapshotInterval>100000</DBRMSnapshotInterval> <DBRMSnapshotInterval>100000</DBRMSnapshotInterval>
<ExternalCriticalThreshold>90</ExternalCriticalThreshold>
<ExternalMajorThreshold>80</ExternalMajorThreshold>
<ExternalMinorThreshold>70</ExternalMinorThreshold>
<TransactionArchivePeriod>10</TransactionArchivePeriod>
<NMSIPAddress>0.0.0.0</NMSIPAddress>
<TempSaveSize>128M</TempSaveSize> <!-- default SWSDL max element save size -->
<WaitPeriod>10</WaitPeriod> <!-- in seconds --> <WaitPeriod>10</WaitPeriod> <!-- in seconds -->
<ProcessRestartCount>10</ProcessRestartCount>
<ProcessRestartPeriod>120</ProcessRestartPeriod>
<SwapAction>restartSystem</SwapAction> <!-- OAM command (or 'none') to run when swap space exceeds Major Threshold -->
<ActivePmFailoverDisabled>n</ActivePmFailoverDisabled>
<MemoryCheckPercent>95</MemoryCheckPercent> <!-- Max real memory to limit growth of buffers to --> <MemoryCheckPercent>95</MemoryCheckPercent> <!-- Max real memory to limit growth of buffers to -->
<DataFileLog>OFF</DataFileLog> <DataFileLog>OFF</DataFileLog>
<!-- enable if you want to limit how much memory may be used for hdfs read/write memory buffers. <!-- enable if you want to limit how much memory may be used for hdfs read/write memory buffers.
<hdfsRdwrBufferMaxSize>8G</hdfsRdwrBufferMaxSize> <hdfsRdwrBufferMaxSize>8G</hdfsRdwrBufferMaxSize>
--> -->
<hdfsRdwrScratch>/rdwrscratch</hdfsRdwrScratch> <!-- Do not set to an hdfs file path --> <hdfsRdwrScratch>/rdwrscratch</hdfsRdwrScratch> <!-- Do not set to an hdfs file path -->
<TempFileDir>/columnstore_tmp_files</TempFileDir>
<!-- Be careful modifying SystemTempFileDir! On start, ExeMgr deletes <!-- Be careful modifying SystemTempFileDir! On start, ExeMgr deletes
the entire subdirectories "joins" & "aggregates" and recreates it to make sure no the entire subdirectories "joins" & "aggregates" and recreates it to make sure no
files are left behind. --> files are left behind. -->
<SystemTempFileDir>/tmp/columnstore_tmp_files</SystemTempFileDir> <SystemTempFileDir>/tmp/columnstore_tmp_files</SystemTempFileDir>
</SystemConfig> </SystemConfig>
<SystemModuleConfig> <SystemModuleConfig>
<ModuleType1>dm</ModuleType1>
<ModuleDesc1>Director Module</ModuleDesc1>
<RunType1>SIMPLEX</RunType1>
<ModuleCount1>0</ModuleCount1>
<ModuleIPAddr1-1-1>0.0.0.0</ModuleIPAddr1-1-1>
<ModuleHostName1-1-1>unassigned</ModuleHostName1-1-1>
<ModuleDisableState1-1>ENABLED</ModuleDisableState1-1>
<ModuleCPUCriticalThreshold1>0</ModuleCPUCriticalThreshold1>
<ModuleCPUMajorThreshold1>0</ModuleCPUMajorThreshold1>
<ModuleCPUMinorThreshold1>0</ModuleCPUMinorThreshold1>
<ModuleCPUMinorClearThreshold1>0</ModuleCPUMinorClearThreshold1>
<ModuleDiskCriticalThreshold1>90</ModuleDiskCriticalThreshold1>
<ModuleDiskMajorThreshold1>80</ModuleDiskMajorThreshold1>
<ModuleDiskMinorThreshold1>70</ModuleDiskMinorThreshold1>
<ModuleMemCriticalThreshold1>90</ModuleMemCriticalThreshold1>
<ModuleMemMajorThreshold1>0</ModuleMemMajorThreshold1>
<ModuleMemMinorThreshold1>0</ModuleMemMinorThreshold1>
<ModuleSwapCriticalThreshold1>90</ModuleSwapCriticalThreshold1>
<ModuleSwapMajorThreshold1>80</ModuleSwapMajorThreshold1>
<ModuleSwapMinorThreshold1>70</ModuleSwapMinorThreshold1>
<ModuleDiskMonitorFileSystem1-1>/</ModuleDiskMonitorFileSystem1-1>
<ModuleDBRootCount1-1>unassigned</ModuleDBRootCount1-1>
<ModuleDBRootID1-1-1>unassigned</ModuleDBRootID1-1-1>
<ModuleType2>um</ModuleType2> <ModuleType2>um</ModuleType2>
<ModuleDesc2>User Module</ModuleDesc2> <ModuleDesc2>User Module</ModuleDesc2>
<RunType2>SIMPLEX</RunType2>
<ModuleCount2>0</ModuleCount2> <ModuleCount2>0</ModuleCount2>
<ModuleIPAddr1-1-2>0.0.0.0</ModuleIPAddr1-1-2> <ModuleIPAddr1-1-2>0.0.0.0</ModuleIPAddr1-1-2>
<ModuleHostName1-1-2>unassigned</ModuleHostName1-1-2> <ModuleHostName1-1-2>unassigned</ModuleHostName1-1-2>
@ -318,7 +96,6 @@
<ModuleDBRootID1-1-2>unassigned</ModuleDBRootID1-1-2> <ModuleDBRootID1-1-2>unassigned</ModuleDBRootID1-1-2>
<ModuleType3>pm</ModuleType3> <ModuleType3>pm</ModuleType3>
<ModuleDesc3>Performance Module</ModuleDesc3> <ModuleDesc3>Performance Module</ModuleDesc3>
<RunType3>SIMPLEX</RunType3>
<ModuleCount3>1</ModuleCount3> <ModuleCount3>1</ModuleCount3>
<ModuleIPAddr1-1-3>127.0.0.1</ModuleIPAddr1-1-3> <ModuleIPAddr1-1-3>127.0.0.1</ModuleIPAddr1-1-3>
<ModuleHostName1-1-3>localhost</ModuleHostName1-1-3> <ModuleHostName1-1-3>localhost</ModuleHostName1-1-3>
@ -340,30 +117,24 @@
<ModuleDBRootCount1-3>1</ModuleDBRootCount1-3> <ModuleDBRootCount1-3>1</ModuleDBRootCount1-3>
<ModuleDBRootID1-1-3>1</ModuleDBRootID1-1-3> <ModuleDBRootID1-1-3>1</ModuleDBRootID1-1-3>
</SystemModuleConfig> </SystemModuleConfig>
<SystemExtDeviceConfig>
<Count>0</Count>
<Name1>unassigned</Name1>
<IPAddr1>0.0.0.0</IPAddr1>
<DisableState1>ENABLED</DisableState1>
</SystemExtDeviceConfig>
<SessionManager> <SessionManager>
<MaxConcurrentTransactions>1000</MaxConcurrentTransactions> <MaxConcurrentTransactions>1000</MaxConcurrentTransactions>
<TxnIDFile>/var/lib/columnstore/data1/systemFiles/dbrm/SMTxnID</TxnIDFile> <TxnIDFile>/var/lib/columnstore/data1/systemFiles/dbrm/SMTxnID</TxnIDFile>
</SessionManager> </SessionManager>
<VersionBuffer> <VersionBuffer>
<!-- VersionBufferFileSize must be a multiple of 8192. <!-- VersionBufferFileSize must be a multiple of 8192.
One version buffer file will be put on each DB root. --> One version buffer file will be put on each DB root. -->
<VersionBufferFileSize>1GB</VersionBufferFileSize> <VersionBufferFileSize>1GB</VersionBufferFileSize>
</VersionBuffer> </VersionBuffer>
<OIDManager> <OIDManager>
<!-- Do not change this file after database built --> <!-- Do not change this file after database built -->
<OIDBitmapFile>/var/lib/columnstore/data1/systemFiles/dbrm/oidbitmap</OIDBitmapFile> <OIDBitmapFile>/var/lib/columnstore/data1/systemFiles/dbrm/oidbitmap</OIDBitmapFile>
<!-- Do not change this value after database built --> <!-- Do not change this value after database built -->
<FirstOID>3000</FirstOID> <FirstOID>3000</FirstOID>
</OIDManager> </OIDManager>
<WriteEngine> <WriteEngine>
<BulkRoot>/var/log/mariadb/columnstore/data/bulk</BulkRoot> <BulkRoot>/var/log/mariadb/columnstore/data/bulk</BulkRoot>
<BulkRollbackDir>/var/lib/columnstore/data1/systemFiles/bulkRollback</BulkRollbackDir> <BulkRollbackDir>/var/lib/columnstore/data1/systemFiles/bulkRollback</BulkRollbackDir>
<MaxFileSystemDiskUsagePct>98</MaxFileSystemDiskUsagePct> <MaxFileSystemDiskUsagePct>98</MaxFileSystemDiskUsagePct>
<CompressedPaddingBlocks>1</CompressedPaddingBlocks> <!-- Number of blocks used to pad compressed chunks --> <CompressedPaddingBlocks>1</CompressedPaddingBlocks> <!-- Number of blocks used to pad compressed chunks -->
</WriteEngine> </WriteEngine>
@ -378,55 +149,10 @@
<Port>8700</Port> <Port>8700</Port>
<Module>pm1</Module> <Module>pm1</Module>
</DBRM_Worker1> </DBRM_Worker1>
<DBRM_Worker2>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker2>
<DBRM_Worker3>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker3>
<DBRM_Worker4>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker4>
<DBRM_Worker5>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker5>
<DBRM_Worker6>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker6>
<DBRM_Worker7>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker7>
<DBRM_Worker8>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker8>
<DBRM_Worker9>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker9>
<DBRM_Worker10>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker10>
<DBBC> <DBBC>
<!-- The percentage of RAM to use for the disk block cache. Defaults to 70% --> <!-- The percentage of RAM to use for the disk block cache. Defaults to 70% -->
<!-- Alternatively, this can be specified in absolute terms using <!-- Alternatively, this can be specified in absolute terms using
the suffixes 'm' or 'g' to denote size in megabytes or gigabytes.--> the suffixes 'm' or 'g' to denote size in megabytes or gigabytes.-->
<!-- <NumBlocksPct>95</NumBlocksPct> --> <!-- <NumBlocksPct>95</NumBlocksPct> -->
<!-- <NumThreads>16</NumThreads> --> <!-- 1-256. Default is 16. --> <!-- <NumThreads>16</NumThreads> --> <!-- 1-256. Default is 16. -->
<NumCaches>1</NumCaches><!-- # of parallel caches to instantiate --> <NumCaches>1</NumCaches><!-- # of parallel caches to instantiate -->
@ -439,40 +165,13 @@
<NumBlocksPct>50</NumBlocksPct> <NumBlocksPct>50</NumBlocksPct>
</DBBC> </DBBC>
<Installation> <Installation>
<SystemStartupOffline>n</SystemStartupOffline>
<InitialInstallFlag>y</InitialInstallFlag>
<SingleServerInstall>y</SingleServerInstall>
<ServerTypeInstall>2</ServerTypeInstall> <ServerTypeInstall>2</ServerTypeInstall>
<PMwithUM>y</PMwithUM> <PMwithUM>y</PMwithUM>
<MySQLRep>n</MySQLRep> <MySQLRep>n</MySQLRep>
<DBRootStorageType>internal</DBRootStorageType> <DBRootStorageType>internal</DBRootStorageType>
<UMStorageType>internal</UMStorageType> <UMStorageType>internal</UMStorageType>
<SystemLogConfigFile>/etc/rsyslog.d/49-columnstore.conf</SystemLogConfigFile>
<Cloud>unassigned</Cloud>
<AmazonVPCNextPrivateIP>autoassign</AmazonVPCNextPrivateIP>
<UMInstanceType>unassigned</UMInstanceType>
<UMSecurityGroup>unassigned</UMSecurityGroup>
<UMVolumeSize>unassigned</UMVolumeSize>
<UMVolumeType>gp2</UMVolumeType>
<UMVolumeIOPS>unassigned</UMVolumeIOPS>
<PMInstanceType>unassigned</PMInstanceType>
<PMVolumeSize>unassigned</PMVolumeSize>
<PMVolumeType>gp2</PMVolumeType>
<PMVolumeIOPS>unassigned</PMVolumeIOPS>
<AmazonPMFailover>y</AmazonPMFailover>
<AmazonAutoTagging>y</AmazonAutoTagging>
<AmazonElasticIPCount>0</AmazonElasticIPCount>
<AmazonElasticIPAddr1>unassigned</AmazonElasticIPAddr1>
<AmazonElasticModule1>unassigned</AmazonElasticModule1>
<DataRedundancyConfig>n</DataRedundancyConfig>
<DataRedundancyCopies>0</DataRedundancyCopies>
<DataRedundancyStorageType>unassigned</DataRedundancyStorageType>
<CoreFileFlag>n</CoreFileFlag>
<MySQLPort>3306</MySQLPort>
<AmazonDeviceName>/dev/xvd</AmazonDeviceName>
<LockFileDirectory>/var/lock/subsys</LockFileDirectory>
<ProfileFile>/etc/profile.d/columnstoreAlias.sh</ProfileFile> <ProfileFile>/etc/profile.d/columnstoreAlias.sh</ProfileFile>
<DataRedundancyNetworkType/> <DataRedundancyNetworkType/>
</Installation> </Installation>
<ExtentMap> <ExtentMap>
<!-- <!--
@ -485,13 +184,12 @@
<HashJoin> <HashJoin>
<MaxBuckets>128</MaxBuckets> <MaxBuckets>128</MaxBuckets>
<MaxElems>128K</MaxElems> <!-- 128 buckets * 128K * 16 = 256 MB --> <MaxElems>128K</MaxElems> <!-- 128 buckets * 128K * 16 = 256 MB -->
<FifoSizeLargeSide>64</FifoSizeLargeSide>
<PmMaxMemorySmallSide>1G</PmMaxMemorySmallSide> <PmMaxMemorySmallSide>1G</PmMaxMemorySmallSide>
<TotalUmMemory>25%</TotalUmMemory> <TotalUmMemory>25%</TotalUmMemory>
<CPUniqueLimit>100</CPUniqueLimit> <CPUniqueLimit>100</CPUniqueLimit>
<AllowDiskBasedJoin>N</AllowDiskBasedJoin> <AllowDiskBasedJoin>N</AllowDiskBasedJoin>
<TempFileCompression>Y</TempFileCompression> <TempFileCompression>Y</TempFileCompression>
<TempFileCompressionType>Snappy</TempFileCompressionType> <!-- LZ4, Snappy --> <TempFileCompressionType>Snappy</TempFileCompressionType> <!-- LZ4, Snappy -->
</HashJoin> </HashJoin>
<JobList> <JobList>
<FlushInterval>16K</FlushInterval> <FlushInterval>16K</FlushInterval>
@ -511,11 +209,6 @@
<!-- <MaxOutstandingRequests>20</MaxOutstandingRequests> --> <!-- <MaxOutstandingRequests>20</MaxOutstandingRequests> -->
<ThreadPoolSize>100</ThreadPoolSize> <ThreadPoolSize>100</ThreadPoolSize>
</JobList> </JobList>
<TupleWSDL>
<MaxSize>1M</MaxSize> <!-- Max size in bytes per bucket -->
<InitialCapacity>1M</InitialCapacity> <!-- Initial capacity per bucket -->
<MaxBuckets>512</MaxBuckets> <!-- Number of buckets -->
</TupleWSDL>
<RowAggregation> <RowAggregation>
<!-- <RowAggrThreads>4</RowAggrThreads> --> <!-- Default value is the number of cores --> <!-- <RowAggrThreads>4</RowAggrThreads> --> <!-- Default value is the number of cores -->
<!-- <RowAggrBuckets>32</RowAggrBuckets> --> <!-- Default value is number of cores * 4 --> <!-- <RowAggrBuckets>32</RowAggrBuckets> --> <!-- Default value is number of cores * 4 -->
@ -540,17 +233,17 @@
</UserPriority> </UserPriority>
<NetworkCompression> <NetworkCompression>
<Enabled>Y</Enabled> <Enabled>Y</Enabled>
<NetworkCompressionType>Snappy</NetworkCompressionType> <!-- LZ4, Snappy --> <NetworkCompressionType>Snappy</NetworkCompressionType> <!-- LZ4, Snappy -->
</NetworkCompression> </NetworkCompression>
<QueryTele> <QueryTele>
<Host>127.0.0.1</Host> <Host>127.0.0.1</Host>
<Port>0</Port> <Port>0</Port>
</QueryTele> </QueryTele>
<StorageManager> <StorageManager>
<MaxSockets>30</MaxSockets> <MaxSockets>30</MaxSockets>
<Enabled>N</Enabled> <Enabled>N</Enabled>
</StorageManager> </StorageManager>
<DataRedundancyConfig> <DataRedundancyConfig>
<DBRoot1PMs/> <DBRoot1PMs/>
</DataRedundancyConfig> </DataRedundancyConfig>
</Columnstore> </Columnstore>

View File

@ -1,547 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<!-- $Id: Columnstore.xml 1286 2009-03-30 22:39:41Z dcathey $ -->
<Columnstore Version="V1.0.0">
<ExeMgr1>
<IPAddr>127.0.0.1</IPAddr>
<Port>8601</Port>
<Module>pm1</Module>
</ExeMgr1>
<JobProc>
<IPAddr>0.0.0.0</IPAddr>
<Port>8602</Port>
</JobProc>
<ProcMgr>
<IPAddr>127.0.0.1</IPAddr>
<Port>8603</Port>
</ProcMgr>
<ProcMgr_Alarm>
<IPAddr>127.0.0.1</IPAddr>
<Port>8606</Port>
</ProcMgr_Alarm>
<ProcStatusControl>
<IPAddr>127.0.0.1</IPAddr>
<Port>8604</Port>
</ProcStatusControl>
<ProcStatusControlStandby>
<IPAddr>0.0.0.0</IPAddr>
<Port>8605</Port>
</ProcStatusControlStandby>
<localhost_ProcessMonitor>
<IPAddr>127.0.0.1</IPAddr>
<Port>8800</Port>
</localhost_ProcessMonitor>
<dm1_ProcessMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8800</Port>
</dm1_ProcessMonitor>
<um1_ProcessMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8800</Port>
</um1_ProcessMonitor>
<pm1_ProcessMonitor>
<IPAddr>127.0.0.1</IPAddr>
<Port>8800</Port>
</pm1_ProcessMonitor>
<dm1_ServerMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8622</Port>
</dm1_ServerMonitor>
<um1_ServerMonitor>
<IPAddr>0.0.0.0</IPAddr>
<Port>8622</Port>
</um1_ServerMonitor>
<pm1_ServerMonitor>
<IPAddr>127.0.0.1</IPAddr>
<Port>8622</Port>
</pm1_ServerMonitor>
<pm1_WriteEngineServer>
<IPAddr>127.0.0.1</IPAddr>
<Port>8630</Port>
</pm1_WriteEngineServer>
<DDLProc>
<IPAddr>127.0.0.1</IPAddr>
<Port>8612</Port>
</DDLProc>
<DMLProc>
<IPAddr>127.0.0.1</IPAddr>
<Port>8614</Port>
</DMLProc>
<BatchInsert>
<RowsPerBatch>10000</RowsPerBatch>
</BatchInsert>
<PrimitiveServers>
<Count>1</Count>
<ConnectionsPerPrimProc>8</ConnectionsPerPrimProc>
<ProcessorThreshold>128</ProcessorThreshold>
<!-- <ProcessorThreads>16</ProcessorThreads> --> <!-- defaults to num cores * 2 -->
<ProcessorQueueSize>10K</ProcessorQueueSize> <!-- minimum of extent size 8192 -->
<DebugLevel>0</DebugLevel>
<LBID_Shift>13</LBID_Shift>
<ColScanBufferSizeBlocks>512</ColScanBufferSizeBlocks>
<ColScanReadAheadBlocks>512</ColScanReadAheadBlocks> <!-- s/b factor of extent size 8192 -->
<!-- <BPPCount>16</BPPCount> --> <!-- Default num cores * 2. A cap on the number of simultaneous primitives per jobstep -->
<PrefetchThreshold>1</PrefetchThreshold>
<PTTrace>0</PTTrace>
<RotatingDestination>n</RotatingDestination>
<!-- <HighPriorityPercentage>60</HighPriorityPercentage> -->
<!-- <MediumPriorityPercentage>30</MediumPriorityPercentage> -->
<!-- <LowPriorityPercentage>10</LowPriorityPercentage> -->
<AggregationMemoryCheck>95</AggregationMemoryCheck>
</PrimitiveServers>
<PMS1>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS1>
<PMS2>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS2>
<PMS3>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS3>
<PMS4>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS4>
<PMS5>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS5>
<PMS6>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS6>
<PMS7>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS7>
<PMS8>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS8>
<PMS9>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS9>
<PMS10>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS10>
<PMS11>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS11>
<PMS12>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS12>
<PMS13>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS13>
<PMS14>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS14>
<PMS15>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS15>
<PMS16>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS16>
<PMS17>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS17>
<PMS18>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS18>
<PMS19>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS19>
<PMS20>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS20>
<PMS21>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS21>
<PMS22>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS22>
<PMS23>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS23>
<PMS24>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS24>
<PMS25>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS25>
<PMS26>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS26>
<PMS27>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS27>
<PMS28>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS28>
<PMS29>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS29>
<PMS30>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS30>
<PMS31>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS31>
<PMS32>
<IPAddr>127.0.0.1</IPAddr>
<Port>8620</Port>
</PMS32>
<SystemConfig>
<SystemLang>C</SystemLang>
<SystemName>columnstore-1</SystemName>
<ParentOAMModuleName>pm1</ParentOAMModuleName>
<StandbyOAMModuleName>unassigned</StandbyOAMModuleName>
<!-- Disabled
<ProcessHeartbeatPeriod>-1</ProcessHeartbeatPeriod>
-->
<!-- Warning: Do not change this value once database is built -->
<DBRootCount>1</DBRootCount>
<DBRoot1>/var/lib/columnstore/data1</DBRoot1>
<DBRMRoot>/var/lib/columnstore/data1/systemFiles/dbrm/BRM_saves</DBRMRoot>
<TableLockSaveFile>/var/lib/columnstore/data1/systemFiles/dbrm/tablelocks</TableLockSaveFile>
<DBRMTimeOut>20</DBRMTimeOut> <!-- in seconds -->
<DBRMSnapshotInterval>100000</DBRMSnapshotInterval>
<ExternalCriticalThreshold>90</ExternalCriticalThreshold>
<ExternalMajorThreshold>80</ExternalMajorThreshold>
<ExternalMinorThreshold>70</ExternalMinorThreshold>
<TempDiskPath>/tmp</TempDiskPath>
<WorkingDir>/tmp</WorkingDir>
<TransactionArchivePeriod>10</TransactionArchivePeriod>
<NMSIPAddress>0.0.0.0</NMSIPAddress>
<TempSaveSize>128M</TempSaveSize> <!-- default SWSDL max element save size -->
<WaitPeriod>10</WaitPeriod> <!-- in seconds -->
<TempFileDir>/tmp/columnstore_tmp_files</TempFileDir>
<ModuleHeartbeatPeriod>10</ModuleHeartbeatPeriod>
<ModuleHeartbeatCount>3</ModuleHeartbeatCount>
<ProcessRestartCount>10</ProcessRestartCount>
<ProcessRestartPeriod>120</ProcessRestartPeriod>
<SwapAction>restartSystem</SwapAction> <!-- OAM command (or 'none') to run when swap space exceeds Major Threshold -->
<ActivePmFailoverDisabled>n</ActivePmFailoverDisabled>
<MemoryCheckPercent>95</MemoryCheckPercent> <!-- Max real memory to limit growth of buffers to -->
<DataFileLog>OFF</DataFileLog>
<!-- enable if you want to limit how much memory may be used for hdfs read/write memory buffers.
<hdfsRdwrBufferMaxSize>8G</hdfsRdwrBufferMaxSize>
-->
<hdfsRdwrScratch>/tmp/rdwrscratch</hdfsRdwrScratch> <!-- Do not set to an hdfs file path -->
<!-- Be careful modifying SystemTempFileDir! On start, ExeMgr deletes
the entire subdirectories "joins" & "aggregates" and recreates it to make sure no
files are left behind. -->
<SystemTempFileDir>/tmp/columnstore_tmp_files</SystemTempFileDir>
</SystemConfig>
<SystemModuleConfig>
<ModuleType1>dm</ModuleType1>
<ModuleDesc1>Director Module</ModuleDesc1>
<RunType1>SIMPLEX</RunType1>
<ModuleCount1>0</ModuleCount1>
<ModuleIPAddr1-1-1>0.0.0.0</ModuleIPAddr1-1-1>
<ModuleHostName1-1-1>unassigned</ModuleHostName1-1-1>
<ModuleDisableState1-1>ENABLED</ModuleDisableState1-1>
<ModuleCPUCriticalThreshold1>0</ModuleCPUCriticalThreshold1>
<ModuleCPUMajorThreshold1>0</ModuleCPUMajorThreshold1>
<ModuleCPUMinorThreshold1>0</ModuleCPUMinorThreshold1>
<ModuleCPUMinorClearThreshold1>0</ModuleCPUMinorClearThreshold1>
<ModuleDiskCriticalThreshold1>90</ModuleDiskCriticalThreshold1>
<ModuleDiskMajorThreshold1>80</ModuleDiskMajorThreshold1>
<ModuleDiskMinorThreshold1>70</ModuleDiskMinorThreshold1>
<ModuleMemCriticalThreshold1>90</ModuleMemCriticalThreshold1>
<ModuleMemMajorThreshold1>0</ModuleMemMajorThreshold1>
<ModuleMemMinorThreshold1>0</ModuleMemMinorThreshold1>
<ModuleSwapCriticalThreshold1>90</ModuleSwapCriticalThreshold1>
<ModuleSwapMajorThreshold1>80</ModuleSwapMajorThreshold1>
<ModuleSwapMinorThreshold1>70</ModuleSwapMinorThreshold1>
<ModuleDiskMonitorFileSystem1-1>/</ModuleDiskMonitorFileSystem1-1>
<ModuleDBRootCount1-1>unassigned</ModuleDBRootCount1-1>
<ModuleDBRootID1-1-1>unassigned</ModuleDBRootID1-1-1>
<ModuleType2>um</ModuleType2>
<ModuleDesc2>User Module</ModuleDesc2>
<RunType2>SIMPLEX</RunType2>
<ModuleCount2>0</ModuleCount2>
<ModuleIPAddr1-1-2>0.0.0.0</ModuleIPAddr1-1-2>
<ModuleHostName1-1-2>unassigned</ModuleHostName1-1-2>
<ModuleDisableState1-2>ENABLED</ModuleDisableState1-2>
<ModuleCPUCriticalThreshold2>0</ModuleCPUCriticalThreshold2>
<ModuleCPUMajorThreshold2>0</ModuleCPUMajorThreshold2>
<ModuleCPUMinorThreshold2>0</ModuleCPUMinorThreshold2>
<ModuleCPUMinorClearThreshold2>0</ModuleCPUMinorClearThreshold2>
<ModuleDiskCriticalThreshold2>90</ModuleDiskCriticalThreshold2>
<ModuleDiskMajorThreshold2>80</ModuleDiskMajorThreshold2>
<ModuleDiskMinorThreshold2>70</ModuleDiskMinorThreshold2>
<ModuleMemCriticalThreshold2>90</ModuleMemCriticalThreshold2>
<ModuleMemMajorThreshold2>0</ModuleMemMajorThreshold2>
<ModuleMemMinorThreshold2>0</ModuleMemMinorThreshold2>
<ModuleSwapCriticalThreshold2>90</ModuleSwapCriticalThreshold2>
<ModuleSwapMajorThreshold2>80</ModuleSwapMajorThreshold2>
<ModuleSwapMinorThreshold2>70</ModuleSwapMinorThreshold2>
<ModuleDiskMonitorFileSystem1-2>/</ModuleDiskMonitorFileSystem1-2>
<ModuleDBRootCount1-2>unassigned</ModuleDBRootCount1-2>
<ModuleDBRootID1-1-2>unassigned</ModuleDBRootID1-1-2>
<ModuleType3>pm</ModuleType3>
<ModuleDesc3>Performance Module</ModuleDesc3>
<RunType3>SIMPLEX</RunType3>
<ModuleCount3>1</ModuleCount3>
<ModuleIPAddr1-1-3>127.0.0.1</ModuleIPAddr1-1-3>
<ModuleHostName1-1-3>localhost</ModuleHostName1-1-3>
<ModuleDisableState1-3>ENABLED</ModuleDisableState1-3>
<ModuleCPUCriticalThreshold3>0</ModuleCPUCriticalThreshold3>
<ModuleCPUMajorThreshold3>0</ModuleCPUMajorThreshold3>
<ModuleCPUMinorThreshold3>0</ModuleCPUMinorThreshold3>
<ModuleCPUMinorClearThreshold3>0</ModuleCPUMinorClearThreshold3>
<ModuleDiskCriticalThreshold3>90</ModuleDiskCriticalThreshold3>
<ModuleDiskMajorThreshold3>80</ModuleDiskMajorThreshold3>
<ModuleDiskMinorThreshold3>70</ModuleDiskMinorThreshold3>
<ModuleMemCriticalThreshold3>90</ModuleMemCriticalThreshold3>
<ModuleMemMajorThreshold3>0</ModuleMemMajorThreshold3>
<ModuleMemMinorThreshold3>0</ModuleMemMinorThreshold3>
<ModuleSwapCriticalThreshold3>90</ModuleSwapCriticalThreshold3>
<ModuleSwapMajorThreshold3>80</ModuleSwapMajorThreshold3>
<ModuleSwapMinorThreshold3>70</ModuleSwapMinorThreshold3>
<ModuleDiskMonitorFileSystem1-3>/</ModuleDiskMonitorFileSystem1-3>
<ModuleDBRootCount1-3>1</ModuleDBRootCount1-3>
<ModuleDBRootID1-1-3>1</ModuleDBRootID1-1-3>
</SystemModuleConfig>
<SystemExtDeviceConfig>
<Count>0</Count>
<Name1>unassigned</Name1>
<IPAddr1>0.0.0.0</IPAddr1>
<DisableState1>ENABLED</DisableState1>
</SystemExtDeviceConfig>
<SessionManager>
<MaxConcurrentTransactions>1000</MaxConcurrentTransactions>
<SharedMemoryTmpFile>/var/lib/columnstore/data1/systemFiles/dbrm/CalpontShm</SharedMemoryTmpFile>
<TxnIDFile>/var/lib/columnstore/data1/systemFiles/dbrm/SMTxnID</TxnIDFile>
</SessionManager>
<SessionMonitor>
<SharedMemoryTmpFile>/var/lib/columnstore/data1/systemFiles/dbrm/CalpontSessionMonitorShm</SharedMemoryTmpFile>
<TransactionAgeLimit>10</TransactionAgeLimit>
</SessionMonitor>
<VersionBuffer>
<!-- VersionBufferFileSize must be a multiple of 8192.
One version buffer file will be put on each DB root. -->
<VersionBufferFileSize>1GB</VersionBufferFileSize>
</VersionBuffer>
<OIDManager>
<!-- XXXPAT: This is located in tmp so we all have R/W access.
It should be relocated to $INSTALLDIR/share/oidbitmap
on the production machine-->
<!-- Do not change this file after database built -->
<OIDBitmapFile>/var/lib/columnstore/data1/systemFiles/dbrm/oidbitmap</OIDBitmapFile>
<!-- Do not change this value after database built -->
<FirstOID>3000</FirstOID>
</OIDManager>
<WriteEngine>
<BulkRoot>/var/lib/columnstore/data/bulk</BulkRoot>
<BulkRollbackDir>/var/lib/columnstore/data/bulk/rollback</BulkRollbackDir>
<MaxFileSystemDiskUsagePct>98</MaxFileSystemDiskUsagePct>
<CompressedPaddingBlocks>1</CompressedPaddingBlocks> <!-- Number of blocks used to pad compressed chunks -->
</WriteEngine>
<DBRM_Controller>
<NumWorkers>1</NumWorkers>
<IPAddr>127.0.0.1</IPAddr>
<Port>8616</Port>
</DBRM_Controller>
<!-- Worker Port: 8700 - 8720 is reserved to support External Modules-->
<DBRM_Worker1>
<IPAddr>127.0.0.1</IPAddr>
<Port>8700</Port>
<Module>pm1</Module>
</DBRM_Worker1>
<DBRM_Worker2>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker2>
<DBRM_Worker3>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker3>
<DBRM_Worker4>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker4>
<DBRM_Worker5>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker5>
<DBRM_Worker6>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker6>
<DBRM_Worker7>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker7>
<DBRM_Worker8>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker8>
<DBRM_Worker9>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker9>
<DBRM_Worker10>
<IPAddr>0.0.0.0</IPAddr>
<Port>8700</Port>
<Module>unassigned</Module>
</DBRM_Worker10>
<DBBC>
<!-- The percentage of RAM to use for the disk block cache. Defaults to 70% -->
<!-- Alternatively, this can be specified in absolute terms using
the suffixes 'm' or 'g' to denote size in megabytes or gigabytes.-->
<NumBlocksPct>50</NumBlocksPct>
<!-- <NumThreads>16</NumThreads> --> <!-- 1-256. Default is 16. -->
<NumCaches>1</NumCaches><!-- # of parallel caches to instantiate -->
<IOMTracing>0</IOMTracing>
<BRPTracing>0</BRPTracing>
<ReportFrequency>65536</ReportFrequency>
<MaxOpenFiles>2K</MaxOpenFiles>
<DecreaseOpenFilesCount>200</DecreaseOpenFilesCount>
<FDCacheTrace>0</FDCacheTrace>
</DBBC>
<Installation>
<SystemStartupOffline>n</SystemStartupOffline>
<InitialInstallFlag>y</InitialInstallFlag>
<SingleServerInstall>y</SingleServerInstall>
<ServerTypeInstall>2</ServerTypeInstall>
<PMwithUM>n</PMwithUM>
<MySQLRep>n</MySQLRep>
<DBRootStorageType>internal</DBRootStorageType>
<UMStorageType>internal</UMStorageType>
<SystemLogConfigFile>unassigned</SystemLogConfigFile>
<rpw>unassigned</rpw>
<Cloud>unassigned</Cloud>
<EEPackageType>rpm</EEPackageType>
<AmazonAccessKey>unassigned</AmazonAccessKey>
<AmazonSecretKey>unassigned</AmazonSecretKey>
<AmazonRegion>us-east-1</AmazonRegion>
<AmazonZone>unassigned</AmazonZone>
<AmazonSubNetID>unassigned</AmazonSubNetID>
<AmazonVPCNextPrivateIP>unassigned</AmazonVPCNextPrivateIP>
<PMInstanceType>unassigned</PMInstanceType>
<UMInstanceType>unassigned</UMInstanceType>
<UMSecurityGroup>unassigned</UMSecurityGroup>
<UMVolumeSize>unassigned</UMVolumeSize>
<PMVolumeSize>unassigned</PMVolumeSize>
<AmazonPMFailover>n</AmazonPMFailover>
<AmazonAutoTagging>n</AmazonAutoTagging>
<AmazonElasticIPCount>0</AmazonElasticIPCount>
<AmazonElasticIPAddr1>unassigned</AmazonElasticIPAddr1>
<AmazonElasticModule1>unassigned</AmazonElasticModule1>
<CoreFileFlag>n</CoreFileFlag>
<JavaHome>unassigned</JavaHome>
<JavaPath>unassigned</JavaPath>
<MySQLPort>3306</MySQLPort>
<LockFileDirectory>/var/lock/subsys</LockFileDirectory>
</Installation>
<ExtentMap>
<!--
WARNING: these can only be changed on an empty system. Once any object has been allocated
it cannot be changed!. Valid values for ExtentRows are: 64M, 8M and 1M. The default is 8M.
-->
<FilesPerColumnPartition>4</FilesPerColumnPartition> <!-- should be multiple of DBRootCount -->
<BRM_UID>0x0</BRM_UID>
</ExtentMap>
<HashJoin>
<MaxBuckets>128</MaxBuckets>
<MaxElems>128K</MaxElems> <!-- 128 buckets * 128K * 16 = 256 MB -->
<FifoSizeLargeSide>64</FifoSizeLargeSide>
<PmMaxMemorySmallSide>1G</PmMaxMemorySmallSide>
<TotalUmMemory>25%</TotalUmMemory>
<TotalPmUmMemory>10%</TotalPmUmMemory>
<CPUniqueLimit>100</CPUniqueLimit>
<AllowDiskBasedJoin>N</AllowDiskBasedJoin>
<TempFileCompression>Y</TempFileCompression>
</HashJoin>
<JobList>
<FlushInterval>16K</FlushInterval>
<FifoSize>16</FifoSize>
<RequestSize>1</RequestSize> <!-- Number of extents per request, should be
less than MaxOutstandingRequests. Otherwise, default value 1 is used. -->
<!-- ProcessorThreadsPerScan is the number of jobs issued to process
each extent. The default is 16. MaxOutstandingRequests is the size of
the window of work in terms of extents. A value of 20 means there
is 20 extents worth of work for the PMs to process at any given time.
ProcessorThreadsPerScan * MaxOutstandingRequests should be at least
as many threads are available across all PMs. -->
<!-- <ProcessorThreadsPerScan>16</ProcessorThreadsPerScan> -->
<!-- MaxOutstandingRequests is going to default to the num of cores available
across all performance modules * 4 divided by the ProcessorThreadsPerScan,
but will be lower bounded by 20 -->
<!-- <MaxOutstandingRequests>20</MaxOutstandingRequests> -->
<ThreadPoolSize>100</ThreadPoolSize>
</JobList>
<TupleWSDL>
<MaxSize>1M</MaxSize> <!-- Max size in bytes per bucket -->
<InitialCapacity>1M</InitialCapacity> <!-- Initial capacity per bucket -->
<MaxBuckets>512</MaxBuckets> <!-- Number of buckets -->
</TupleWSDL>
<RowAggregation>
<!-- <RowAggrThreads>4</RowAggrThreads> --> <!-- Default value is the number of cores -->
<!-- <RowAggrBuckets>32</RowAggrBuckets> --> <!-- Default value is number of cores * 4 -->
<!-- <RowAggrRowGroupsPerThread>20</RowAggrRowGroupsPerThread> --> <!-- Default value is 20 -->
<!-- <AllowDiskBasedAggregation>N</AllowDiskBasedAggregation> --> <!-- Default value is N -->
<!-- <Compression>SNAPPY</Compression> --> <!-- Disabled by default -->
</RowAggregation>
<CrossEngineSupport>
<Host>127.0.0.1</Host>
<Port>3306</Port>
<User>root</User>
<Password></Password>
<TLSCA></TLSCA>
<TLSClientCert></TLSClientCert>
<TLSClientKey></TLSClientKey>
</CrossEngineSupport>
<QueryStats>
<Enabled>N</Enabled>
</QueryStats>
<UserPriority>
<Enabled>N</Enabled>
</UserPriority>
<NetworkCompression>
<Enabled>Y</Enabled>
</NetworkCompression>
<QueryTele>
<Host>127.0.0.1</Host>
<Port>0</Port>
</QueryTele>
<StorageManager>
<MaxSockets>30</MaxSockets>
<Enabled>N</Enabled>
</StorageManager>
</Columnstore>

View File

@ -167,7 +167,6 @@ void Oam::getSystemConfig(const std::string& moduletype, ModuleTypeConfig& modul
const string Section = "SystemModuleConfig"; const string Section = "SystemModuleConfig";
const string MODULE_TYPE = "ModuleType"; const string MODULE_TYPE = "ModuleType";
const string MODULE_DESC = "ModuleDesc"; const string MODULE_DESC = "ModuleDesc";
const string MODULE_RUN_TYPE = "RunType";
const string MODULE_COUNT = "ModuleCount"; const string MODULE_COUNT = "ModuleCount";
const string MODULE_DISABLE_STATE = "ModuleDisableState"; const string MODULE_DISABLE_STATE = "ModuleDisableState";
const string MODULE_CPU_CRITICAL = "ModuleCPUCriticalThreshold"; const string MODULE_CPU_CRITICAL = "ModuleCPUCriticalThreshold";
@ -198,7 +197,6 @@ void Oam::getSystemConfig(const std::string& moduletype, ModuleTypeConfig& modul
string ModuleCount = MODULE_COUNT + itoa(moduleTypeID); string ModuleCount = MODULE_COUNT + itoa(moduleTypeID);
string ModuleType = MODULE_TYPE + itoa(moduleTypeID); string ModuleType = MODULE_TYPE + itoa(moduleTypeID);
string ModuleDesc = MODULE_DESC + itoa(moduleTypeID); string ModuleDesc = MODULE_DESC + itoa(moduleTypeID);
string ModuleRunType = MODULE_RUN_TYPE + itoa(moduleTypeID);
string ModuleCPUCriticalThreshold = MODULE_CPU_CRITICAL + itoa(moduleTypeID); string ModuleCPUCriticalThreshold = MODULE_CPU_CRITICAL + itoa(moduleTypeID);
string ModuleCPUMajorThreshold = MODULE_CPU_MAJOR + itoa(moduleTypeID); string ModuleCPUMajorThreshold = MODULE_CPU_MAJOR + itoa(moduleTypeID);
string ModuleCPUMinorThreshold = MODULE_CPU_MINOR + itoa(moduleTypeID); string ModuleCPUMinorThreshold = MODULE_CPU_MINOR + itoa(moduleTypeID);
@ -216,7 +214,6 @@ void Oam::getSystemConfig(const std::string& moduletype, ModuleTypeConfig& modul
moduletypeconfig.ModuleCount = strtol(sysConfig->getConfig(Section, ModuleCount).c_str(), 0, 0); moduletypeconfig.ModuleCount = strtol(sysConfig->getConfig(Section, ModuleCount).c_str(), 0, 0);
moduletypeconfig.ModuleType = sysConfig->getConfig(Section, ModuleType); moduletypeconfig.ModuleType = sysConfig->getConfig(Section, ModuleType);
moduletypeconfig.ModuleDesc = sysConfig->getConfig(Section, ModuleDesc); moduletypeconfig.ModuleDesc = sysConfig->getConfig(Section, ModuleDesc);
moduletypeconfig.RunType = sysConfig->getConfig(Section, ModuleRunType);
moduletypeconfig.ModuleCPUCriticalThreshold = moduletypeconfig.ModuleCPUCriticalThreshold =
strtol(sysConfig->getConfig(Section, ModuleCPUCriticalThreshold).c_str(), 0, 0); strtol(sysConfig->getConfig(Section, ModuleCPUCriticalThreshold).c_str(), 0, 0);
moduletypeconfig.ModuleCPUMajorThreshold = moduletypeconfig.ModuleCPUMajorThreshold =

View File

@ -115,7 +115,6 @@ const std::string UnassignedName = "unassigned";
const std::string configSections[] = {"SystemConfig", const std::string configSections[] = {"SystemConfig",
"SystemModuleConfig", "SystemModuleConfig",
"SystemModuleConfig", "SystemModuleConfig",
"SystemExtDeviceConfig",
"SessionManager", "SessionManager",
"VersionBuffer", "VersionBuffer",
"OIDManager", "OIDManager",
@ -235,7 +234,6 @@ struct ModuleTypeConfig_s
{ {
std::string ModuleType; //!< Module Type std::string ModuleType; //!< Module Type
std::string ModuleDesc; //!< Module Description std::string ModuleDesc; //!< Module Description
std::string RunType; //!< Run Type
uint16_t ModuleCount; //!< Module Equipage Count uint16_t ModuleCount; //!< Module Equipage Count
uint16_t ModuleCPUCriticalThreshold; //!< CPU Critical Threahold % uint16_t ModuleCPUCriticalThreshold; //!< CPU Critical Threahold %
uint16_t ModuleCPUMajorThreshold; //!< CPU Major Threahold % uint16_t ModuleCPUMajorThreshold; //!< CPU Major Threahold %

View File

@ -62,7 +62,6 @@ string rootPassword = "";
string debug_flag = "0"; string debug_flag = "0";
string mysqlpw = " "; string mysqlpw = " ";
string tmpDir; string tmpDir;
string ProfileFile;
int runningThreads = 0; int runningThreads = 0;
pthread_mutex_t mutex1 = PTHREAD_MUTEX_INITIALIZER; pthread_mutex_t mutex1 = PTHREAD_MUTEX_INITIALIZER;
@ -121,9 +120,8 @@ void* childReportThread(threadInfo_t* st)
cout << "Get " + reportType + " report data for " + remoteModuleName + " " << endl; cout << "Get " + reportType + " report data for " + remoteModuleName + " " << endl;
string cmd = "remote_command.sh " + remoteModuleIP + " " + rootPassword + " '. " + ProfileFile + ";" + string cmd = "remote_command.sh " + remoteModuleIP + " " + rootPassword + ";" + reportType + "Report.sh " +
reportType + "Report.sh " + remoteModuleName + "' " + debug_flag + " - forcetty"; remoteModuleName + "' " + debug_flag + " - forcetty";
int rtnCode = system(cmd.c_str()); int rtnCode = system(cmd.c_str());
if (WEXITSTATUS(rtnCode) != 0) if (WEXITSTATUS(rtnCode) != 0)
@ -333,7 +331,7 @@ int main(int argc, char* argv[])
} }
// get Local Module Name and Server Install Indicator // get Local Module Name and Server Install Indicator
string singleServerInstall; string singleServerInstall = "n";
oamModuleInfo_t st; oamModuleInfo_t st;
@ -348,15 +346,6 @@ int main(int argc, char* argv[])
exit(-1); exit(-1);
} }
try
{
oam.getSystemConfig("SingleServerInstall", singleServerInstall);
}
catch (...)
{
singleServerInstall = "y";
}
if (argc == 1) if (argc == 1)
{ {
argv[1] = &helpArg[0]; argv[1] = &helpArg[0];
@ -592,17 +581,6 @@ int main(int argc, char* argv[])
exit(-1); exit(-1);
} }
// Get Profile file
try
{
ProfileFile = sysConfig->getConfig(InstallSection, "ProfileFile");
}
catch (...)
{
cout << "ERROR: Problem getting ProfileFile" << endl;
exit(-1);
}
string ModuleSection = "SystemModuleConfig"; string ModuleSection = "SystemModuleConfig";
for (unsigned int i = 0; i < sysModuleTypeConfig.moduletypeconfig.size(); i++) for (unsigned int i = 0; i < sysModuleTypeConfig.moduletypeconfig.size(); i++)

View File

@ -160,7 +160,6 @@ void getModuleTypeConfig(FILE* pOutputFile)
fprintf(pOutputFile, "ModuleType '%s' Configuration information\n", moduletype.c_str()); fprintf(pOutputFile, "ModuleType '%s' Configuration information\n", moduletype.c_str());
fprintf(pOutputFile, "ModuleDesc = %s\n", fprintf(pOutputFile, "ModuleDesc = %s\n",
systemmoduletypeconfig.moduletypeconfig[i].ModuleDesc.c_str()); systemmoduletypeconfig.moduletypeconfig[i].ModuleDesc.c_str());
fprintf(pOutputFile, "RunType = %s\n", systemmoduletypeconfig.moduletypeconfig[i].RunType.c_str());
fprintf(pOutputFile, "ModuleCount = %i\n", moduleCount); fprintf(pOutputFile, "ModuleCount = %i\n", moduleCount);
if (moduleCount > 0) if (moduleCount > 0)
@ -379,23 +378,16 @@ void getStorageConfig(FILE* pOutputFile)
string volumeName = oam::UnassignedName; string volumeName = oam::UnassignedName;
string deviceNameID = "PMVolumeDeviceName" + oam.itoa(*pt); string deviceNameID = "PMVolumeDeviceName" + oam.itoa(*pt);
string deviceName = oam::UnassignedName; string deviceName = oam::UnassignedName;
string amazonDeviceNameID = "PMVolumeAmazonDeviceName" + oam.itoa(*pt);
string amazondeviceName = oam::UnassignedName;
try try
{ {
oam.getSystemConfig(volumeNameID, volumeName); oam.getSystemConfig(volumeNameID, volumeName);
oam.getSystemConfig(deviceNameID, deviceName); oam.getSystemConfig(deviceNameID, deviceName);
oam.getSystemConfig(amazonDeviceNameID, amazondeviceName);
} }
catch (...) catch (...)
{ {
continue; continue;
} }
fprintf(pOutputFile,
"Amazon EC2 Volume Name/Device Name/Amazon Device Name for DBRoot%u: %s, %s, %s", *pt,
volumeName.c_str(), deviceName.c_str(), amazondeviceName.c_str());
} }
} }
catch (exception& e) catch (exception& e)
@ -412,22 +404,16 @@ void getStorageConfig(FILE* pOutputFile)
string volumeName = oam::UnassignedName; string volumeName = oam::UnassignedName;
string deviceNameID = "PMVolumeDeviceName" + oam.itoa(*pt1); string deviceNameID = "PMVolumeDeviceName" + oam.itoa(*pt1);
string deviceName = oam::UnassignedName; string deviceName = oam::UnassignedName;
string amazonDeviceNameID = "PMVolumeAmazonDeviceName" + oam.itoa(*pt1);
string amazondeviceName = oam::UnassignedName;
try try
{ {
oam.getSystemConfig( volumeNameID, volumeName); oam.getSystemConfig( volumeNameID, volumeName);
oam.getSystemConfig( deviceNameID, deviceName); oam.getSystemConfig( deviceNameID, deviceName);
oam.getSystemConfig( amazonDeviceNameID, amazondeviceName);
} }
catch (...) catch (...)
{ {
continue; continue;
} }
fprintf(pOutputFile,"Amazon EC2 Volume Name/Device Name/Amazon Device Name for DBRoot%u: %s, %s,
%s",*pt1,volumeName.c_str(),deviceName.c_str(),amazondeviceName.c_str());
}*/ }*/
} }

View File

@ -27,7 +27,6 @@
// Copyright: See COPYING file that comes with this distribution // Copyright: See COPYING file that comes with this distribution
// //
// //
#include "primitivemsg.h"
#include "bytestream.h" #include "bytestream.h"
#include "messagequeue.h" #include "messagequeue.h"
#include "serializeable.h" #include "serializeable.h"

View File

@ -117,7 +117,6 @@ void DictStep::prep(int8_t outputType, bool makeAbsRids)
primMsg->ism.Interleave = 0; primMsg->ism.Interleave = 0;
primMsg->ism.Flags = 0; primMsg->ism.Flags = 0;
// primMsg->ism.Flags = PrimitiveMsg::planFlagsToPrimFlags(traceFlags);
primMsg->ism.Command = DICT_SIGNATURE; primMsg->ism.Command = DICT_SIGNATURE;
primMsg->ism.Size = bufferSize; primMsg->ism.Size = bufferSize;
primMsg->ism.Type = 2; primMsg->ism.Type = 2;

View File

@ -601,7 +601,6 @@ int ServicePrimProc::Child()
if (temp >= 0) if (temp >= 0)
maxPct = temp; maxPct = temp;
// @bug4507, configurable pm aggregation AggregationMemoryCheck
// We could use this same mechanism for other growing buffers. // We could use this same mechanism for other growing buffers.
int aggPct = 95; int aggPct = 95;
temp = toInt(cf->getConfig("SystemConfig", "MemoryCheckPercent")); temp = toInt(cf->getConfig("SystemConfig", "MemoryCheckPercent"));

View File

@ -119,7 +119,6 @@ void UmSocketSelector::loadUMModuleInfo()
std::cout << "ModuleConfig for type: " << UM_MODTYPE << std::endl; std::cout << "ModuleConfig for type: " << UM_MODTYPE << std::endl;
std::cout << "ModuleDesc = " << moduleTypeConfig.ModuleDesc << std::endl; std::cout << "ModuleDesc = " << moduleTypeConfig.ModuleDesc << std::endl;
std::cout << "ModuleCount = " << moduleCount << std::endl; std::cout << "ModuleCount = " << moduleCount << std::endl;
std::cout << "RunType = " << moduleTypeConfig.RunType << std::endl;
#endif #endif
if (moduleCount > 0) if (moduleCount > 0)

View File

@ -65,7 +65,6 @@ class WOConfigFileTest : public CppUnit::TestFixture
void test1() void test1()
{ {
WriteOnceConfig woc(cf); WriteOnceConfig woc(cf);
CPPUNIT_ASSERT(woc.owns("PrimitiveServers", "LBID_Shift"));
CPPUNIT_ASSERT(woc.owns("SystemConfig", "DBRootCount")); CPPUNIT_ASSERT(woc.owns("SystemConfig", "DBRootCount"));
CPPUNIT_ASSERT(woc.owns("SystemConfig", "DBRMRoot")); CPPUNIT_ASSERT(woc.owns("SystemConfig", "DBRMRoot"));
@ -73,9 +72,6 @@ class WOConfigFileTest : public CppUnit::TestFixture
int vali; int vali;
vali = Config::fromText(woc.getConfig("PrimitiveServers", "LBID_Shift"));
CPPUNIT_ASSERT(vali == 13);
woc.setConfig("SystemConfig", "DBRootCount", "10"); woc.setConfig("SystemConfig", "DBRootCount", "10");
vali = Config::fromText(woc.getConfig("SystemConfig", "DBRootCount")); vali = Config::fromText(woc.getConfig("SystemConfig", "DBRootCount"));
CPPUNIT_ASSERT(vali == 10); CPPUNIT_ASSERT(vali == 10);

View File

@ -258,22 +258,6 @@ int fixupCalpontXML()
cout << "Fixing " << section << "." << parm << " = " << val << endl; cout << "Fixing " << section << "." << parm << " = " << val << endl;
} }
// Fixup WES
section = "pm1_WriteEngineServer";
parm = "IPAddr";
val = cf->getConfig(section, parm);
if (val.empty())
{
val = "127.0.0.1";
cf->setConfig(section, parm, val);
cout << "Adding " << section << "." << parm << " = " << val << endl;
parm = "Port";
val = "8630";
cf->setConfig(section, parm, val);
cout << "Adding " << section << "." << parm << " = " << val << endl;
}
// Fixup TableLockSaveFile // Fixup TableLockSaveFile
section = "SystemConfig"; section = "SystemConfig";
parm = "TableLockSaveFile"; parm = "TableLockSaveFile";