You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by su...@apache.org on 2017/05/31 18:47:47 UTC

[46/50] [abbrv] incubator-trafodion git commit: Fixed persistent process restart on node up. Fixed rmscheck, missing code from last merge. Changed sqgen to source in node section from 'sqconfig.file' and persist section from 'sqconfig.persist' file and c

Fixed persistent process restart on node up.
Fixed rmscheck, missing code from last merge.
Changed sqgen to source in node section from 'sqconfig.file' and persist section from 'sqconfig.persist' file and changed bash and and python install scripts to account for this change.
Fixed logic to account for name change of TMID process.


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/363df1f9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/363df1f9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/363df1f9

Branch: refs/heads/master
Commit: 363df1f97c2e081a38a8f9c3ab106b583b2948b6
Parents: 23bb89a
Author: Zalo Correa <za...@esgyn.com>
Authored: Tue May 16 17:14:54 2017 -0700
Committer: Zalo Correa <za...@esgyn.com>
Committed: Tue May 16 17:14:54 2017 -0700

----------------------------------------------------------------------
 core/sqf/monitor/linux/clusterconf.cxx          |   1 +
 core/sqf/monitor/linux/lnodeconfig.cxx          |   3 +-
 core/sqf/monitor/linux/lnodeconfig.h            |   2 +
 core/sqf/monitor/linux/makefile                 |   4 +
 core/sqf/monitor/linux/persistconfig.cxx        |  78 +++
 core/sqf/monitor/linux/persistconfig.h          |   7 +-
 core/sqf/monitor/linux/pstartd.cxx              | 599 +++++++------------
 core/sqf/monitor/linux/pstartd.h                |  34 +-
 core/sqf/monitor/linux/trafconf.cxx             |   8 +
 core/sqf/sql/scripts/gensq.pl                   |  22 +-
 core/sqf/sql/scripts/sqconfig                   |  68 +--
 core/sqf/sql/scripts/sqgen                      |   2 +-
 .../src/seatrans/tm/hbasetmlib2/idtmclicom.h    |  14 +-
 core/sqf/src/tm/Makefile                        |   4 +
 install/installer/addNode_step2                 |   6 -
 install/installer/traf_sqconfig                 |   4 -
 .../python-installer/scripts/traf_sqconfig.py   |   4 -
 17 files changed, 345 insertions(+), 515 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/clusterconf.cxx
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/clusterconf.cxx b/core/sqf/monitor/linux/clusterconf.cxx
index 74dd69d..1316b02 100644
--- a/core/sqf/monitor/linux/clusterconf.cxx
+++ b/core/sqf/monitor/linux/clusterconf.cxx
@@ -453,6 +453,7 @@ bool CClusterConfig::LoadPersistConfig( void )
     for (pkit = pkeysVector.begin(); pkit < pkeysVector.end(); pkit++ )
     {
         memset( &persistConfig, 0, sizeof(persist_configuration_t) );
+        memset( &persistConfigInfo, 0, sizeof(persistConfigInfo_t) );
         strncpy( persistConfig.persist_prefix
                , pkit->c_str()
                , sizeof(persistConfig.persist_prefix));

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/lnodeconfig.cxx
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/lnodeconfig.cxx b/core/sqf/monitor/linux/lnodeconfig.cxx
index 34df543..77c2360 100644
--- a/core/sqf/monitor/linux/lnodeconfig.cxx
+++ b/core/sqf/monitor/linux/lnodeconfig.cxx
@@ -50,6 +50,7 @@ CLNodeConfig::CLNodeConfig( CPNodeConfig *pnodeConfig
                           , lnodeConfigInfo_t &lnodeConfigInfo
                           )
             : nid_(lnodeConfigInfo.nid)
+            , zid_(pnodeConfig->GetPNid())
             , coreMask_(lnodeConfigInfo.coreMask)
             , firstCore_(lnodeConfigInfo.firstCore)
             , lastCore_(lnodeConfigInfo.lastCore)
@@ -86,7 +87,7 @@ const char *CLNodeConfig::GetName( void )
 
 int  CLNodeConfig::GetPNid( void ) 
 {
-    return( pnodeConfig_->GetPNid( ) );
+    return( pnodeConfig_->GetPNid() );
 }
 
 CLNodeConfigContainer::CLNodeConfigContainer( void )

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/lnodeconfig.h
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/lnodeconfig.h b/core/sqf/monitor/linux/lnodeconfig.h
index 37185c8..e32c08f 100644
--- a/core/sqf/monitor/linux/lnodeconfig.h
+++ b/core/sqf/monitor/linux/lnodeconfig.h
@@ -92,6 +92,7 @@ public:
     inline CLNodeConfig *GetNext( void ) { return( next_); }
     inline CLNodeConfig *GetNextP( void ) { return( nextP_); }
     inline int           GetNid( void ) { return( nid_ ); }
+    inline int           GetZid( void ) { return( zid_ ); }
     int                  GetPNid( void );
     CPNodeConfig        *GetPNodeConfig( void ) { return(pnodeConfig_); }
 
@@ -101,6 +102,7 @@ public:
 protected:
 private:
     int           nid_;         // Logical Node Identifier
+    int           zid_;         // Zone Identifier
     cpu_set_t     coreMask_;    // mask of SMP processor cores used by logical node
     int           firstCore_;   // First SMP processor core used by logical node
     int           lastCore_;    // Last SMP processor core used by logical node

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/makefile
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/makefile b/core/sqf/monitor/linux/makefile
index a933bdc..b8c034a 100644
--- a/core/sqf/monitor/linux/makefile
+++ b/core/sqf/monitor/linux/makefile
@@ -256,6 +256,10 @@ PSTARTDOBJS += $(OUTDIR)/lock.o
 PSTARTDOBJS += $(OUTDIR)/monlogging.o
 PSTARTDOBJS += $(OUTDIR)/verspstartd.o
 PSTARTDOBJS += $(OUTDIR)/CommonLogger.o
+PSTARTDOBJS += $(OUTDIR)/clusterconf.o
+PSTARTDOBJS += $(OUTDIR)/lnodeconfig.o
+PSTARTDOBJS += $(OUTDIR)/pnodeconfig.o
+PSTARTDOBJS += $(OUTDIR)/persistconfig.o
 
 MEMLOGOBJS = $(OUTDIR)/monmemlog.o
 MEMLOGOBJS += $(OUTDIR)/versmemlog.o

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/persistconfig.cxx
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/persistconfig.cxx b/core/sqf/monitor/linux/persistconfig.cxx
index 485c6a4..c7572c3 100644
--- a/core/sqf/monitor/linux/persistconfig.cxx
+++ b/core/sqf/monitor/linux/persistconfig.cxx
@@ -191,8 +191,10 @@ const char *FormatZidString( FormatZid_t type )
 
 CPersistConfig::CPersistConfig( persistConfigInfo_t &persistConfigInfo )
               : persistPrefix_(persistConfigInfo.persistPrefix)
+              , processName_("")
               , processNamePrefix_(persistConfigInfo.processNamePrefix)
               , processNameFormat_(persistConfigInfo.processNameFormat)
+              , stdoutFile_("")
               , stdoutPrefix_(persistConfigInfo.stdoutPrefix)
               , stdoutFormat_(persistConfigInfo.stdoutFormat)
               , programName_(persistConfigInfo.programName)
@@ -294,6 +296,44 @@ const char *CPersistConfig::GetProcessName( int nid )
     return( processName_.c_str() );
 }
 
+const char *CPersistConfig::GetStdoutFile( int nid )
+{
+    const char method_name[] = "CPersistConfig::GetStdoutFile";
+    TRACE_ENTRY;
+
+    char nidStr[MAX_PROCESS_NAME];
+
+    switch (stdoutNidFormat_)
+    {
+    case Nid_ALL:
+    case Nid_RELATIVE:
+        if (nid == -1)
+        {
+            stdoutFile_ = stdoutPrefix_;
+        }
+        else
+        {
+            sprintf( nidStr, "%d", nid );
+            stdoutFile_ = stdoutPrefix_ + nidStr;
+        }
+        break;
+    case Nid_Undefined:
+        stdoutFile_ = stdoutPrefix_;
+    }
+
+    if ( trace_settings & (TRACE_REQUEST | TRACE_PROCESS))
+    {
+        trace_printf( "%s@%d stdout prefix=%s, file=%s, format=%s\n"
+                    , method_name, __LINE__
+                    , stdoutPrefix_.c_str()
+                    , stdoutFile_.c_str()
+                    , FormatNidString(stdoutNidFormat_));
+    }
+
+    TRACE_EXIT;
+    return( stdoutFile_.c_str() );
+}
+
 bool CPersistConfig::IsPersistConfig( const char *processName, int nid )
 {
     const char method_name[] = "CPersistConfig:IsPersistConfig";
@@ -311,6 +351,44 @@ bool CPersistConfig::IsPersistConfig( const char *processName, int nid )
     return( match );
 }
 
+bool CPersistConfig::IsZoneMatch( int zid )
+{
+    const char method_name[] = "CPersistConfig:IsZoneMatch";
+    TRACE_ENTRY;
+    
+    bool match = false;
+
+    switch (stdoutNidFormat_)
+    {
+    case Zid_ALL:
+        if (zid == -1)
+        {
+            match = false;
+        }
+        else
+        {
+            match = true;
+        }
+        break;
+    case Zid_RELATIVE:
+        if (zid == -1)
+        {
+            match = false;
+        }
+        else
+        {
+            match = true;
+        }
+        break;
+    case Zid_Undefined:
+        match = true;
+        break;
+    }
+
+    TRACE_EXIT;
+    return( match );
+}
+
 CPersistConfigContainer::CPersistConfigContainer( void )
                        : persistsCount_(0)
                        , head_(NULL)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/persistconfig.h
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/persistconfig.h b/core/sqf/monitor/linux/persistconfig.h
index 0614b7e..1a5ac43 100644
--- a/core/sqf/monitor/linux/persistconfig.h
+++ b/core/sqf/monitor/linux/persistconfig.h
@@ -110,10 +110,11 @@ public:
     inline CPersistConfig *GetNext( void ){ return( next_); }
 
     inline const char   *GetPersistPrefix( void ) { return( persistPrefix_.c_str() ); }
-    const char          *GetProcessName( int nid );
+           const char   *GetProcessName( int nid );
     inline const char   *GetProcessNamePrefix( void ) { return( processNamePrefix_.c_str() ); }
     inline const char   *GetProcessNameFormat( void ) { return( processNameFormat_.c_str() ); }
     inline FormatNid_t   GetProcessNameNidFormat( void ) { return( processNameNidFormat_ ); }
+           const char   *GetStdoutFile( int nid );
     inline const char   *GetStdoutPrefix( void ) { return( stdoutPrefix_.c_str() ); }
     inline const char   *GetStdoutFormat( void ) { return( stdoutFormat_.c_str() ); }
     inline FormatNid_t   GetStdoutNidFormat( void ) { return( stdoutNidFormat_ ); }
@@ -124,7 +125,8 @@ public:
     inline bool          GetRequiresDTM( void ) { return ( requiresDTM_ ); }
     inline int           GetPersistRetries( void ) { return ( persistRetries_ ); }
     inline int           GetPersistWindow( void ) { return ( persistWindow_ ); }
-    inline bool          IsPersistConfig( const char *processName, int nid );
+           bool          IsPersistConfig( const char *processName, int nid );
+           bool          IsZoneMatch( int zid );
 
 protected:
 private:
@@ -132,6 +134,7 @@ private:
     string          processName_;
     string          processNamePrefix_;
     string          processNameFormat_;
+    string          stdoutFile_;
     string          stdoutPrefix_;
     string          stdoutFormat_;
     string          programName_;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/pstartd.cxx
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/pstartd.cxx b/core/sqf/monitor/linux/pstartd.cxx
index 63ac28e..f3388d8 100644
--- a/core/sqf/monitor/linux/pstartd.cxx
+++ b/core/sqf/monitor/linux/pstartd.cxx
@@ -34,6 +34,7 @@
 #include "msgdef.h"
 #include "seabed/trace.h"
 #include "montrace.h"
+#include "clusterconf.h"
 #include "pstartd.h"
 
 
@@ -42,6 +43,7 @@ char ga_ms_su_c_port[MPI_MAX_PORT_NAME] = {0}; // connection port - not used
 
 long trace_settings = 0;
 int MyPNID = -1;
+int MyZid = -1;
 int MyNid = -1;
 int MyPid = -1;
 int gv_ms_su_nid = -1;          // Local IO nid to make compatible w/ Seabed
@@ -50,6 +52,7 @@ SB_Verif_Type  gv_ms_su_verif = -1;
 bool tracing = false;
 bool shuttingDown = false;
 
+CClusterConfig ClusterConfig; // Trafodion Configuration objects
 CMonUtil monUtil;
 CPStartD *pStartD;
 CMonLog *MonLog = NULL;
@@ -57,6 +60,8 @@ CMonLog *MonLog = NULL;
 DEFINE_EXTERN_COMP_DOVERS(pstartd)
 DEFINE_EXTERN_COMP_PRINTVERS(pstartd)
 
+const char *ProcessTypeString( PROCESSTYPE type );
+
 const char *MessageTypeString( MSGTYPE type )
 {
     const char *str = NULL;
@@ -136,65 +141,6 @@ const char *MessageTypeString( MSGTYPE type )
     return( str );
 }
 
-const char *ProcessTypeString( PROCESSTYPE type )
-{
-    const char *str;
-
-    switch( type )
-    {
-        case ProcessType_TSE:
-            str = "TSE";
-            break;
-        case ProcessType_DTM:
-            str = "DTM";
-            break;
-        case ProcessType_ASE:
-            str = "ASE";
-            break;
-        case ProcessType_Generic:
-            str = "Generic";
-            break;
-        case ProcessType_Watchdog:
-            str = "Watchdog";
-            break;
-        case ProcessType_AMP:
-            str = "AMP";
-            break;
-        case ProcessType_Backout:
-            str = "Backout";
-            break;
-        case ProcessType_VolumeRecovery:
-            str = "VolumeRecovery";
-            break;
-        case ProcessType_MXOSRVR:
-            str = "MXOSRVR";
-            break;
-        case ProcessType_SPX:
-            str = "SPX";
-            break;
-        case ProcessType_SSMP:
-            str = "SSMP";
-            break;
-        case ProcessType_PSD:
-            str = "PSD";
-            break;
-        case ProcessType_SMS:
-            str = "SMS";
-            break;
-        case ProcessType_TMID:
-            str = "TMID";
-            break;
-        case ProcessType_PERSIST:
-            str = "PERSIST";
-            break;
-        default:
-            str = "Undefined";
-            break;
-    }
-
-    return( str );
-}
-
 void InitLocalIO( void )
 {
     gp_local_mon_io = new Local_IO_To_Monitor( -1 );
@@ -252,13 +198,6 @@ void localIONoticeCallback(struct message_def *recv_msg, int )
                           method_name, __LINE__, recv_msg->u.request.u.up.nid,
                           recv_msg->u.request.u.up.node_name);
         }
-        CNodeUpReq * reqNodeUp;
-        reqNodeUp = new CNodeUpReq(recv_msg->u.request.u.up.nid,
-                                   recv_msg->u.request.u.up.node_name,
-                                   true);
-
-        pStartD->enqueueReq( reqNodeUp );
-        pStartD->CLock::wakeOne();
         break;
 
     case MsgType_ProcessDeath:
@@ -860,59 +799,58 @@ void CMonUtil::processArgs( int argc, char *argv[] )
     pnid_ = atoi(argv[2]);
     nid_ = atoi(argv[3]);
     pid_ = atoi(argv[4]);
+    zid_ = atoi(argv[8]);
     gv_ms_su_verif  = verifier_ = atoi(argv[9]);
 
     strncpy( processName_, argv[5], sizeof(processName_) );
     processName_[sizeof(processName_)-1] = '\0';
 }
 
+CNodeUpReq::CNodeUpReq(int nid, char nodeName[], bool requiresDTM)
+          : nid_(nid)
+          , requiresDTM_(requiresDTM)
+    { 
+        strncpy(nodeName_, nodeName, sizeof(nodeName_));
+        nodeName_[sizeof(nodeName_)-1] = '\0';
+
+        CLNodeConfig *lnodeConfig = ClusterConfig.GetLNodeConfig( nid );
+        zid_ = lnodeConfig->GetZid();
+    }
+
 void CNodeUpReq::performRequest()
 {
     const char method_name[] = "CNodeUpReq::performRequest";
 
     char buf[MON_STRING_BUF_SIZE];
-    snprintf( buf, sizeof(buf), "Received 'Node Up' event for node %d, "
-              "requires DTM flag=%d\n", nid_, requiresDTM_);
+    snprintf( buf, sizeof(buf)
+            , "Received 'Node Up' event for node %d, "
+              "requires DTM flag=%d\n"
+            , nid_, requiresDTM_);
     monproc_log_write( MONUTIL_PERFORM_REQUEST_1, SQ_LOG_INFO, buf );
 
-    //    [ todo: need to check if nid_ is any one of the logical nodes in
-    //      the physical node ]
-    if ( nid_ == MyPNID )
+    if ( zid_ == MyZid )
     {
         if ( tracing )
         {
             trace_printf("%s@%d invoking startProcs(%d, %d)\n",
                          method_name, __LINE__, nid_, requiresDTM_);
         }
-        pStartD->startProcs(nid_, requiresDTM_);
+        pStartD->startProcs( requiresDTM_ );
     }
     else
     {
         if ( tracing )
         {
-            trace_printf("%s@%d Ignoring node up for for node %d (%s), my node is %d\n", method_name, __LINE__, nid_, nodeName_, MyPNID );
+            trace_printf( "%s@%d Ignoring node up for for node %s, nid=%d, "
+                          ", zid=%d, MyZid=%d\n"
+                        , method_name, __LINE__
+                        , nodeName_, nid_, zid_, MyZid );
         }
     }
 }
 
 CPStartD::CPStartD()
-         :trafConfigInitialized_(false)
 {
-    const char method_name[] = "CPStartD::CPStartD";
-
-    int rc = tc_initialize( tracing );
-    if ( rc )
-    {
-        char buf[MON_STRING_BUF_SIZE];
-        snprintf( buf, sizeof(buf)
-                , "[%s], Can't initialize configuration!\n"
-                , method_name );
-        monproc_log_write( PSTARTD_PSTARTD_1, SQ_LOG_CRIT, buf );
-    }
-    else
-    {
-        trafConfigInitialized_ = true;
-    }
 }
 
 CPStartD::~CPStartD()
@@ -942,359 +880,243 @@ void CPStartD::enqueueReq(CRequest * req)
     workQ_.push_back ( req );
 }
 
-void CPStartD::waitForEvent( void )
+bool CPStartD::loadConfiguration( void )
 {
-    CAutoLock autoLock(getLocker());
-    wait();
-}
+    if ( ClusterConfig.IsConfigReady() )
+    {
+        // It was previously loaded, remove the old configuration
+        ClusterConfig.Clear();
+    }
 
+    if ( ClusterConfig.Initialize() )
+    {
+        if ( ! ClusterConfig.LoadConfig() )
+        {
+            printf("[%s], Failed to load cluster configuration.\n", MyName);
+            abort();
+        }
+    }
+    else
+    {
+        printf( "[%s] Warning: No cluster.conf found\n",MyName);
+        abort();
+    }
+
+    return true;
+}
 
-void CPStartD::startProcess( const char *pName
-                           , const char *prefix
-                           , persist_configuration_t &persistConfig )
+void CPStartD::startProcess( CPersistConfig *persistConfig )
 {
     const char method_name[] = "CPStartD::startProcess";
 
-    PROCESSTYPE progType = ProcessType_Undefined;
+    PROCESSTYPE procType = ProcessType_Undefined;
     int progArgC = 0;
+    string procName;
     string progArgs;
     string progStdout;
     string progProgram;
     char newProcName[MAX_PROCESS_PATH];
-    int progNid = MyNid;
     bool result;
     int newNid;
     int newPid;
-    int okMask = 0;
     int argBegin[MAX_ARGS];
     int argLen[MAX_ARGS];
 
-    string value = persistConfig.process_type;
-    okMask |= 0x1;
-    if (value.compare("DTM") == 0)
-        progType = ProcessType_DTM;
-    else if (value.compare("GENERIC") == 0)
-        progType = ProcessType_Generic;
-    else if (value.compare("PERSIST") == 0)
-        progType = ProcessType_PERSIST;
-    else if (value.compare("PSD") == 0)
-        progType = ProcessType_PSD;
-    else if (value.compare("SPX") == 0)
-        progType = ProcessType_SPX;
-    else if (value.compare("SSMP") == 0)
-        progType = ProcessType_SSMP;
-    else if (value.compare("SMS") == 0)
-        progType = ProcessType_SMS;
-    else if (value.compare("TMID") == 0)
-        progType = ProcessType_TMID;
-    else if (value.compare("WDG") == 0)
-        progType = ProcessType_Watchdog;
-
-    okMask |= 0x2;
-    progStdout = persistConfig.std_out;
-    okMask |= 0x4;
-    progProgram = persistConfig.program_name;
-
-    if ( okMask & 0x7 )
-    {
-        if ( tracing )
-        {
-            trace_printf("%s@%d Will start process: nid=%d, type=%s, name=%s, "
-                         "prog=%s, stdout=%s, argc=%d, args=%s\n",
-                         method_name, __LINE__, progNid,
-                         ProcessTypeString(progType), pName, progProgram.c_str(),
-                         progStdout.c_str(), progArgC, progArgs.c_str());
-        }
+    procName = persistConfig->GetProcessName( MyNid );
+    procType = persistConfig->GetProcessType();
+    progStdout = persistConfig->GetStdoutFile( MyNid );
+    progProgram = persistConfig->GetProgramName();
 
-        char buf[MON_STRING_BUF_SIZE];
-        snprintf( buf, sizeof(buf), "Starting process %s on nid=%d, program="
-                  "%s, type=%d\n", pName, progNid, progProgram.c_str(), progType);
-        monproc_log_write( PSTARTD_START_PROCESS_1, SQ_LOG_INFO, buf );
-
-        result = monUtil.requestNewProcess(progNid,
-                                           progType, pName,
-                                           progProgram.c_str(), "",
-                                           progStdout.c_str(),
-                                           progArgC, progArgs.c_str(),
-                                           argBegin, argLen,
-                                           newNid, newPid, newProcName);
-        if ( tracing )
-        {
-            trace_printf("%s@%d requestNewProcess returned: %d\n",
-                         method_name, __LINE__, result);
-        }
-    }
-
-}
-
-bool CPStartD::seapilotDisabled ( void )
-{
-    const char method_name[] = "CPStartD::seapilotDisabled";
-
-    bool disabled = false;
-
-    struct Get_reply_def * regData;
-    monUtil.requestGet (ConfigType_Cluster, "", "SQ_SEAPILOT_SUSPENDED", false,
-                        regData);
-    if ( regData->num_returned == 1 )
+    if ( tracing )
     {
-        disabled = strcmp(regData->list[0].value, "1") == 0;
+        trace_printf( "%s@%d Will start process: nid=%d, type=%s, name=%s, "
+                      "prog=%s, stdout=%s, argc=%d, args=%s\n"
+                    , method_name, __LINE__, MyNid
+                    , ProcessTypeString(procType), procName.c_str()
+                    , progProgram.c_str(), progStdout.c_str()
+                    , progArgC, progArgs.c_str());
     }
+
+    char buf[MON_STRING_BUF_SIZE];
+    snprintf( buf, sizeof(buf)
+            , "Starting process %s on nid=%d, program=%s, type=%s\n"
+            , procName.c_str(), MyNid, progProgram.c_str()
+            , ProcessTypeString(procType));
+    monproc_log_write( PSTARTD_START_PROCESS_1, SQ_LOG_INFO, buf );
+
+    result = monUtil.requestNewProcess( MyNid
+                                      , procType
+                                      , procName.c_str()
+                                      , progProgram.c_str()
+                                      , ""
+                                      , progStdout.c_str()
+                                      , progArgC
+                                      , progArgs.c_str()
+                                      , argBegin
+                                      , argLen
+                                      , newNid
+                                      , newPid
+                                      , newProcName);
     if ( tracing )
     {
-        trace_printf("%s@%d regData->num_returned=%d, regData->list[0].value=%s\n", method_name, __LINE__, regData->num_returned, regData->list[0].value);
-        trace_printf("%s@%d Registry: seapilotDisabled=%d\n",
-                     method_name, __LINE__, disabled);
+        trace_printf("%s@%d requestNewProcess returned: %d\n",
+                     method_name, __LINE__, result);
     }
-    free(regData);
-
-    return disabled;
 }
 
-void CPStartD::startProcs ( int nid, bool requiresDTM )
+void CPStartD::startProcs ( bool requiresDTM )
 {
     const char method_name[] = "CPStartD::startProcs";
 
     /*
-1. cache configuation database to find all persistent data.
-2. for each persistent process:
-   a) ask monitor if process is currently running
-   b) if not running, start it on the logical node using process
-      definition from the database.
+     1. use persist configuration objects
+     2. for each persistent configuration object:
+        a) determine if the process type is candidate for restart in this nid
+        b) ask monitor if process is currently running in this nid
+        c) if not running, start it
     */
 
-    list<pair<string,string> > procsToStart;
-    list<string> prefixToStart;
-    list<string> keys;
-    map<string,string> persistDataMap;
-    persist_configuration_t persistConfig;
+    bool foundConfig = false;
+    CPersistConfig *persistConfig;
+    string procName = "";
     
-
-    // Get persistent process keys
-    int rc;
-    char persistProcessKeys[TC_PERSIST_KEYS_VALUE_MAX];
-    rc = tc_get_persist_keys( persistProcessKeys );
-    if ( rc )
+    persistConfig = ClusterConfig.GetFirstPersistConfig();
+    if (persistConfig)
     {
-        char buf[MON_STRING_BUF_SIZE];
-        snprintf( buf, sizeof(buf)
-                , "[%s] Persist keys configuration does not exist!\n"
-                , method_name );
-        monproc_log_write( PSTARTD_STARTPROCS_1, SQ_LOG_CRIT, buf );
-
-        return; // no keys, no work
-    }
-
-    if ( strlen( persistProcessKeys ) )
-    {
-        processKeys( persistProcessKeys, keys );
-    }
-
-    // Get persistent process configuration for each key
-    list<string>::iterator keyIt;
-    for (keyIt = keys.begin(); keyIt != keys.end(); ++keyIt)
-    {
-        string procName = "";
-        string procType = "";
-        string zones = "";
-        string prefix = (*keyIt);
-        rc = tc_get_persist_process( prefix.c_str(), &persistConfig );
-        if ( rc )
-        {
-            char buf[MON_STRING_BUF_SIZE];
-            snprintf( buf, sizeof(buf)
-                    , "[%s] Persist configuration for %s does not exist!\n"
-                    , method_name, prefix.c_str() );
-            monproc_log_write( PSTARTD_STARTPROCS_2, SQ_LOG_ERR, buf );
-            continue;
-        }
-
-        if ( tracing )
-        {
-            trace_printf( "%s@%d Persist Prefix =%s\n"
-                          "\t\tProcess Name    = %s\n"
-                          "\t\tProcess Type    = %s\n"
-                          "\t\tProgram Name    = %s\n"
-                          "\t\tSTDOUT          = %s\n"
-                          "\t\tRequires DTM    = %s\n"
-                          "\t\tPersist Retries = %d\n"
-                          "\t\tPersist Window  = %d\n"
-                          "\t\tPersist Zones   = %s\n"
-                        , method_name, __LINE__
-                        , persistConfig.persist_prefix
-                        , persistConfig.process_name
-                        , persistConfig.process_type
-                        , persistConfig.program_name
-                        , persistConfig.std_out
-                        , persistConfig.requires_DTM ? "Y" : "N"
-                        , persistConfig.persist_retries
-                        , persistConfig.persist_window
-                        , persistConfig.persist_zones );
-        }
-
-        procName = persistConfig.process_name;
-
-        if ( persistConfig.requires_DTM && !requiresDTM )
+        for ( ; persistConfig; persistConfig = persistConfig->GetNext() )
         {
             if ( tracing )
             {
-                trace_printf("%s@%d Persist type %s NOT targeted for restart DTM not ready\n",
-                             method_name, __LINE__, persistConfig.persist_prefix );
+                trace_printf( "%s@%d Persist Prefix =%s\n"
+                              "\t\tProcess Name    = %s\n"
+                              "\t\tProcess Type    = %s\n"
+                              "\t\tProgram Name    = %s\n"
+                              "\t\tSTDOUT          = %s\n"
+                              "\t\tRequires DTM    = %s\n"
+                              "\t\tPersist Retries = %d\n"
+                              "\t\tPersist Window  = %d\n"
+                              "\t\tPersist Zones   = %s\n"
+                            , method_name, __LINE__
+                            , persistConfig->GetPersistPrefix()
+                            , persistConfig->GetProcessName( MyNid )
+                            , ProcessTypeString(persistConfig->GetProcessType())
+                            , persistConfig->GetProgramName()
+                            , persistConfig->GetStdoutFile( MyNid )
+                            , persistConfig->GetRequiresDTM() ? "Y" : "N"
+                            , persistConfig->GetPersistRetries()
+                            , persistConfig->GetPersistWindow()
+                            , persistConfig->GetZoneFormat() );
             }
-            continue;
-        }
-        else if ( persistConfig.requires_DTM && requiresDTM )
-        {
-            if ( tracing )
+    
+            switch (persistConfig->GetProcessType())
             {
-                trace_printf("%s@%d Persist type %s NOT targeted for restart DTM ready\n",
-                             method_name, __LINE__, persistConfig.persist_prefix );
-            }
-            continue;
-        }
-
-        procType = persistConfig.process_type;
-        zones    = persistConfig.persist_zones;
+            case ProcessType_TMID:
+            case ProcessType_PERSIST:
+            case ProcessType_SSMP:
+                if ( persistConfig->GetRequiresDTM() && !requiresDTM )
+                {
+                    if ( tracing )
+                    {
+                        trace_printf("%s@%d Persist type %s NOT targeted for restart DTM not ready\n",
+                                     method_name, __LINE__, persistConfig->GetPersistPrefix() );
+                    }
+                    break;
+                }
+                else if ( !persistConfig->GetRequiresDTM() && requiresDTM )
+                {
+                    if ( tracing )
+                    {
+                        trace_printf("%s@%d Persist type %s NOT targeted for restart DTM ready\n",
+                                     method_name, __LINE__, persistConfig->GetPersistPrefix() );
+                    }
+                    break;
+                }
 
-        if ( tracing )
-        {
-            trace_printf("%s@%d Persist %s process type %s targeted for restart\n",
-                         method_name, __LINE__,
-                         prefix.c_str(), persistConfig.process_type );
-        }
+                procName = persistConfig->GetProcessName( MyNid );
         
-
-        if ((procName.length() != 0) && (zones.length() != 0))
-        {
-            int procNid = -1;
-            int procPid = -1;
-
-            if (zoneMatch(zones.c_str()))
-            {
-                if (!monUtil.requestProcInfo(procName.c_str(), procNid, procPid))
-                {   // Save this process name
-                    procsToStart.push_back(pair<string,string>(procName, prefix));
+                if ( tracing )
+                {
+                    trace_printf( "%s@%d Persist %s process type %s targeted for restart\n"
+                                , method_name, __LINE__
+                                , persistConfig->GetPersistPrefix()
+                                , ProcessTypeString(persistConfig->GetProcessType()) );
                 }
-                else
+                
+                if (procName.length() != 0)
                 {
-                    if ( procNid != -1)
+                    int procNid = -1;
+                    int procPid = -1;
+        
+                    if (persistConfig->IsZoneMatch( MyZid ))
                     {
-                        char buf[MON_STRING_BUF_SIZE];
-                        snprintf( buf, sizeof(buf), "Not starting process %s "
-                                  "because it is already running\n",
-                                  procName.c_str());
-                        monproc_log_write( PSTARTD_STARTPROCS_3, SQ_LOG_INFO, buf );
-    
-                        if ( tracing )
+                        if ( ! monUtil.requestProcInfo( procName.c_str()
+                                                      , procNid
+                                                      , procPid))
+                        {   // Process does not exist, so start it
+                            startProcess( persistConfig );
+                        }
+                        else
                         {
-                            trace_printf("%s@%d %s", method_name, __LINE__,
-                                         buf);
+                            if ( procNid != -1)
+                            {
+                                char buf[MON_STRING_BUF_SIZE];
+                                snprintf( buf, sizeof(buf), "Not starting process %s "
+                                          "because it is already running\n",
+                                          procName.c_str());
+                                monproc_log_write( PSTARTD_STARTPROCS_1, SQ_LOG_INFO, buf );
+            
+                                if ( tracing )
+                                {
+                                    trace_printf( "%s@%d %s"
+                                                , method_name, __LINE__, buf);
+                                }
+                            }
                         }
                     }
+                    else
+                    {
+                        char buf[MON_STRING_BUF_SIZE];
+                        snprintf( buf, sizeof(buf)
+                                , "[%s] Zone does not match for persist type %s"
+                                  ", zone format=%s, MyZid=%d\n"
+                                , method_name
+                                , persistConfig->GetPersistPrefix()
+                                , persistConfig->GetZoneFormat()
+                                , MyZid );
+                        monproc_log_write( PSTARTD_STARTPROCS_2, SQ_LOG_CRIT, buf );
+                    }
                 }
+                break;
+            case ProcessType_DTM:
+            case ProcessType_PSD:
+            case ProcessType_Watchdog:
+            default:
+                // Skip these, they are managed by DTM Lead and monitor processes
+                if ( tracing )
+                {
+                    trace_printf("%s@%d Persist type %s NOT targeted for restart\n",
+                                 method_name, __LINE__, persistConfig->GetPersistPrefix() );
+                }
+                break;
             }
         }
     }
-
-    list<pair<string,string> >::iterator it;
-    for ( it = procsToStart.begin(); it != procsToStart.end(); ++it)
-    {
-        const char * procName = (*it).first.c_str();
-        const char * prefix = (*it).second.c_str();
-
-        if ( tracing )
-        {
-            trace_printf("%s@%d Will start process %s for zone %d\n",
-                         method_name, __LINE__, procName, nid);
-        }
-        startProcess( procName, prefix, persistConfig );
-    }
-    procsToStart.clear();
-}
-
-void CPStartD::processKeys(const char *keys, list<string> &keyList)
-{
-    char *keyDup = strdup(keys);
-    char *k = keyDup;
-    for (;;)
-    {
-        char *kComma = index(k, ',');
-        if (kComma == NULL)
-        {
-            keyList.push_back(k);
-            break;
-        }
-        else
-        {
-            *kComma = '\0';
-            keyList.push_back(k);
-            k = &kComma[1];
-        }
-    }
-    free(keyDup);
-}
-
-void CPStartD::replaceNid(char *str)
-{
-    for (;;)
+    if (!foundConfig)
     {
-        //                     1234
-        char *p = strstr(str, "%nid");
-        if (p == NULL)
-            break;
-        char tail[1000];
-        if (p[4] == '+')
-            strcpy(tail, &p[5]);
-        else
-            strcpy(tail, &p[4]);
-        sprintf(p, "%d", MyNid);
-        strcat(p, tail);
+        printf ("[%s] Persistent process configuration does not exist\n", MyName);
     }
 }
 
-void CPStartD::replaceZid(char *str)
-{
-    for (;;) {
-        //                     1234
-        char *p = strstr(str, "%zid");
-        if (p == NULL)
-            break;
-        char tail[1000];
-        if (p[4] == '+')
-            strcpy(tail, &p[5]);
-        else
-            strcpy(tail, &p[4]);
-        sprintf(p, "%d", MyNid);
-        strcat(p, tail);
-    }
-}
-
-bool CPStartD::zoneMatch ( const char *zones )
+void CPStartD::waitForEvent( void )
 {
-    bool ret;
-    int zone;
-    const char *z = zones;
-    for (;;)
-    {
-        const char *zComma = index(z, ',');
-        if (zComma == NULL)
-        {
-            sscanf(z, "%d", &zone);
-            ret = (zone == MyNid);
-            break;
-        }
-        else
-        {
-            sscanf(z, "%d", &zone);
-            ret = (zone == MyNid);
-            if (ret)
-                 break;
-            z = &zComma[1];
-        }
-    }
-    return ret;
+    CAutoLock autoLock(getLocker());
+    wait();
 }
 
+//
+// To enable tracing, add the following to sql/scripts/mon.env file:
+// PSD_TRACE=1
+//
 void TraceInit( int & argc, char **& argv )
 {
     char traceFileName[MAX_PROCESS_PATH];
@@ -1389,6 +1211,7 @@ int main (int argc, char *argv[])
     monUtil.processArgs (argc, argv);
     MyName = monUtil.getProcName();
     gv_ms_su_nid = MyPNID = monUtil.getPNid();
+    MyZid = monUtil.getZid();
     MyNid = monUtil.getNid();
     MyPid = monUtil.getPid();
 
@@ -1396,6 +1219,12 @@ int main (int argc, char *argv[])
 
     pStartD = new CPStartD;
 
+    if ( !pStartD->loadConfiguration() )
+    {
+        trace_printf("%s@%d Exiting!\n",  method_name, __LINE__);
+        exit (1);
+    }
+
     InitLocalIO( );
 
     gp_local_mon_io->set_cb(localIONoticeCallback, "notice");

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/pstartd.h
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/pstartd.h b/core/sqf/monitor/linux/pstartd.h
index 81c60ed..e697638 100644
--- a/core/sqf/monitor/linux/pstartd.h
+++ b/core/sqf/monitor/linux/pstartd.h
@@ -46,6 +46,7 @@ class CMonUtil
     int getNid() { return nid_; }
     int getPid() { return pid_; }
     int getPNid() { return pnid_; }
+    int getZid() { return zid_; }
     const char *getProcName() { return processName_; }
     bool getTrace() { return trace_; }
     int getVerifier() { return verifier_; }
@@ -80,6 +81,7 @@ class CMonUtil
  private:
     char   processName_[MAX_PROCESS_PATH];   // current process name
     int    pnid_;          // current process physical node id
+    int    zid_;           // current process node id
     int    nid_;           // current process node id
     int    pid_;           // current process process id
     Verifier_t verifier_;  // current process verifier
@@ -99,18 +101,14 @@ class CRequest
 class CNodeUpReq: public CRequest
 {
  public:
-    CNodeUpReq(int nid, char nodeName[], bool requiresDTM)
-        : nid_(nid), requiresDTM_(requiresDTM)
-    { 
-        strncpy(nodeName_, nodeName, sizeof(nodeName_));
-        nodeName_[sizeof(nodeName_)-1] = '\0';
-    }
+    CNodeUpReq(int nid, char nodeName[], bool requiresDTM);
     virtual ~CNodeUpReq() {}
 
     void performRequest();
 
  private:
     int  nid_;
+    int  zid_;
     bool requiresDTM_;
     char nodeName_[MPI_MAX_PROCESSOR_NAME];
 };
@@ -133,32 +131,16 @@ class CPStartD : public CLock
     typedef enum { NodeUp } pStartD_t;
 
     void enqueueReq(CRequest * req);
-
     CRequest * getReq( void );
-
     int getReqCount( void );
-
-    void waitForEvent( void ) ;
-
-    void startProcess( const char *pName
-                     , const char *prefix
-                     , persist_configuration_t &persistConfig );
-
-    void startProcs ( int nid, bool requiresDTM );
-
-    bool seapilotDisabled ( void );
-
-    bool zoneMatch ( const char *zones );
-
+    bool loadConfiguration( void );
+    void startProcess( CPersistConfig *persistConfig );
+    void startProcs ( bool requiresDTM );
+    void waitForEvent( void );
 
  private:
-    void processKeys(const char *keys, list<string> &keyList);
-    void replaceNid(char *str);
-    void replaceZid(char *str);
 
     list<CRequest *>  workQ_;
-
-    bool            trafConfigInitialized_;
 };
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/monitor/linux/trafconf.cxx
----------------------------------------------------------------------
diff --git a/core/sqf/monitor/linux/trafconf.cxx b/core/sqf/monitor/linux/trafconf.cxx
index 7128967..9fff70a 100644
--- a/core/sqf/monitor/linux/trafconf.cxx
+++ b/core/sqf/monitor/linux/trafconf.cxx
@@ -40,6 +40,7 @@ using namespace std;
 #include <mpi.h>
 
 #include "msgdef.h"
+#include "montrace.h"
 #include "sqevlog/evl_sqlog_writer.h"
 #include "clusterconf.h"
 
@@ -590,6 +591,13 @@ int main( int argc, char *argv[] )
         }
     }
 
+    char *env;
+    env = getenv("TC_TRACE_ENABLE");
+    if ( env && *env == '1' )
+    {
+        trace_settings |= TRACE_TRAFCONFIG;
+    }
+
     if ( !ClusterConfig.Initialize() )
     {
         printf( "Failed to initialize 'sqconfig.db'!\n" );

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/sql/scripts/gensq.pl
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/gensq.pl b/core/sqf/sql/scripts/gensq.pl
index ffeddaf..3bc8ae4 100755
--- a/core/sqf/sql/scripts/gensq.pl
+++ b/core/sqf/sql/scripts/gensq.pl
@@ -436,7 +436,7 @@ sub printSQShellCommand {
 sub processNodes {
     my $bNodeSpecified = 0;
 
-    while (<SRC>) {
+    while (<>) {
         next if (/^$/);
         next if (/^#/);
         if (/^_virtualnodes/) {
@@ -502,7 +502,7 @@ sub processNodes {
 
 sub processPersist {
     my $err = 0;
-    while (<SRC>) {
+    while (<>) {
         if (/^begin persist/) {
         }
         elsif (/^end persist/) {
@@ -541,7 +541,7 @@ sub printZoneList {
 }
 
 sub processFloatingIp {
-    while (<SRC>) {
+    while (<>) {
         if (/^process/) {
             @this_line = split(/;/, $_);
         if($#this_line >= 2) {
@@ -632,9 +632,6 @@ sub printInitLinesAuxFiles {
 
 sub openFiles {
 
-    open (SRC,"<$infile")
-        or die("unable to open $infile");
-
     open (SQS,">$coldscriptFileName")
         or die("unable to open $coldscriptFileName");
 
@@ -694,7 +691,6 @@ sub endGame {
     print "Generated SSCP Startup      file: $startSSCP\n";
     print "Generated SSCP Stop         file: $stopSSCP\n";
 
-    close(SRC);
     close(SQS);
     close(SQSH);
 
@@ -731,11 +727,11 @@ sub endGame {
 
 sub doInit {
 
-    $infile=@ARGV[0];
-    $scriptFileName=@ARGV[1];
-    $g_HostName=$ARGV[2];
-    $g_FTFlag=$ARGV[3];
-    $g_PERFFlag=$ARGV[4];
+    $scriptFileName= shift(@ARGV);
+    $g_HostName= shift(@ARGV);
+    $g_FTFlag= shift(@ARGV);
+    $g_PERFFlag= shift(@ARGV);
+    # remaining args are input files
 
 
     $startIDTM="idtmstart";
@@ -769,7 +765,7 @@ doInit();
 openFiles;
 
 
-while (<SRC>) {
+while (<>) {
     if (/^begin node/) {
         processNodes;
         printInitialLines;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/sql/scripts/sqconfig
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/sqconfig b/core/sqf/sql/scripts/sqconfig
index 4de7fd8..93a552f 100644
--- a/core/sqf/sql/scripts/sqconfig
+++ b/core/sqf/sql/scripts/sqconfig
@@ -19,69 +19,7 @@
 #
 # @@@ END COPYRIGHT @@@
 
-begin persist
-
-#
-# The following PROCESS_TYPEs are persistent and have special process management:
-#    DTM
-#    TMID
-#    SSMP
-#    PSD
-#    WDG
-#
-# Generic persistent processes are of PROCESS_TYPE:
-#    PERSIST
-#
-
-PERSIST_PROCESS_KEYS = DTM,TMID,SSCP,SSMP,PSD,WDG
-
-DTM_PROCESS_NAME     = $TM%nid+
-DTM_PROCESS_TYPE     = DTM
-DTM_PROGRAM_NAME     = tm
-DTM_REQUIRES_DTM     = N
-DTM_STDOUT           = stdout_DTM%nid
-DTM_PERSIST_RETRIES  = 2,30
-DTM_PERSIST_ZONES    = %zid
-
-TMID_PROCESS_NAME    = $TMID
-TMID_PROCESS_TYPE    = TMID
-TMID_PROGRAM_NAME    = idtmsrv
-TMID_REQUIRES_DTM    = N
-TMID_STDOUT          = stdout_TMID
-TMID_PERSIST_RETRIES = 2,30
-TMID_PERSIST_ZONES   = %zid+
-
-SSCP_PROCESS_NAME    = $ZSC%nid+
-SSCP_PROCESS_TYPE    = PERSIST
-SSCP_PROGRAM_NAME    = mxsscp
-SSCP_REQUIRES_DTM    = Y
-SSCP_STDOUT          = stdout_ZSC%nid
-SSCP_PERSIST_RETRIES = 10,60
-SSCP_PERSIST_ZONES   = %zid
-
-SSMP_PROCESS_NAME    = $ZSM%nid+
-SSMP_PROCESS_TYPE    = SSMP
-SSMP_PROGRAM_NAME    = mxssmp
-SSMP_REQUIRES_DTM    = Y
-SSMP_STDOUT          = stdout_ZSM%nid
-SSMP_PERSIST_RETRIES = 10,60
-SSMP_PERSIST_ZONES   = %zid
-
-PSD_PROCESS_NAME     = $PSD%nid+
-PSD_PROCESS_TYPE     = PSD
-PSD_PROGRAM_NAME     = pstartd
-PSD_REQUIRES_DTM     = N
-PSD_STDOUT           = stdout_PSD%nid
-PSD_PERSIST_RETRIES  = 10,60
-PSD_PERSIST_ZONES    = %zid
-
-WDG_PROCESS_NAME     = $WDG%nid+
-WDG_PROCESS_TYPE     = WDG
-WDG_PROGRAM_NAME     = sqwatchdog
-WDG_REQUIRES_DTM     = N
-WDG_STDOUT           = stdout_WDG%nid
-WDG_PERSIST_RETRIES  = 10,60
-WDG_PERSIST_ZONES    = %zid
-
-end persist
+begin node
+_virtualnodes 2
+end node
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/sql/scripts/sqgen
----------------------------------------------------------------------
diff --git a/core/sqf/sql/scripts/sqgen b/core/sqf/sql/scripts/sqgen
index fd49ed8..f953717 100755
--- a/core/sqf/sql/scripts/sqgen
+++ b/core/sqf/sql/scripts/sqgen
@@ -231,7 +231,7 @@ sqlite3 -init createConfigDb $SQCONFIG_DB_FILE <<eof
 .quit
 eof
 
-./gensq.pl $SQCONFIG_FILE $SQSCRIPT_FILE `hostname` $FT_FLAG $PERF_FLAG
+./gensq.pl $SQSCRIPT_FILE `hostname` $FT_FLAG $PERF_FLAG $SQCONFIG_FILE $TRAF_HOME/sql/scripts/sqconfig.persist
 sq_stat=$?
 if [[ $sq_stat != 0 ]]; then 
     exit $sq_stat;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/src/seatrans/tm/hbasetmlib2/idtmclicom.h
----------------------------------------------------------------------
diff --git a/core/sqf/src/seatrans/tm/hbasetmlib2/idtmclicom.h b/core/sqf/src/seatrans/tm/hbasetmlib2/idtmclicom.h
index 7cc264e..4823e00 100644
--- a/core/sqf/src/seatrans/tm/hbasetmlib2/idtmclicom.h
+++ b/core/sqf/src/seatrans/tm/hbasetmlib2/idtmclicom.h
@@ -448,14 +448,12 @@ static int do_get_servers2(MS_Mon_Process_Info_Type  *pp_pi,
     lv_scount = 0;
     for (lv_inx = 0; lv_inx < pv_count; lv_inx++) {
         lp_sname = pp_pi[lv_inx].process_name;
-        if (memcmp(lp_sname, "$TSID", 5) == 0) {
-            if (isdigit(lp_sname[5])) {
-                if (gv_verbose)
-                    printf("cli: sname=%s\n", lp_sname);
-                assert(lv_scount < pv_max);
-                ppp_snames[lv_scount] = lp_sname;
-                lv_scount++;
-            }
+        if (memcmp(lp_sname, "$TMID", 5) == 0) {
+            if (gv_verbose)
+                printf("cli: sname=%s\n", lp_sname);
+            assert(lv_scount < pv_max);
+            ppp_snames[lv_scount] = lp_sname;
+            lv_scount++;
         }
     }
     return lv_scount;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/core/sqf/src/tm/Makefile
----------------------------------------------------------------------
diff --git a/core/sqf/src/tm/Makefile b/core/sqf/src/tm/Makefile
index 4151b24..ff5686e 100644
--- a/core/sqf/src/tm/Makefile
+++ b/core/sqf/src/tm/Makefile
@@ -131,6 +131,7 @@ IDSRVOBJS	= $(OUTDIR)/idtmsrv.o \
 ALLOBJS	= $(LIBSTMOBJS) \
 		  $(LIBSXATMOBJS) \
 		  $(TMOBJS) \
+		  $(IDSRVOBJS) \
 		  $(LIBXARMOBJS) \
 
 CFILES		= `ls *.c`
@@ -190,6 +191,9 @@ setup:
 clean:
 	$(RM) $(LIBEXPDIR)/libsxatmlib.so $(LIBEXPDIR)/libxarm.so $(LIBEXPDIR)/libstmlib.so *.gcda *.gcno *.gcov
 	$(RM)  $(BINEXPDIR)/tm
+	$(RM)  $(BINEXPDIR)/idtmsrv
+	$(RM)  $(OUTDIR)/tm
+	$(RM)  $(OUTDIR)/idtmsrv
 	$(RM)  $(ALLOBJS)
 	$(RM)  $(CDEPFILES)
 	cd $(HBASE_TRX_LOC); make clean

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/install/installer/addNode_step2
----------------------------------------------------------------------
diff --git a/install/installer/addNode_step2 b/install/installer/addNode_step2
index 9a28cbb..58df373 100755
--- a/install/installer/addNode_step2
+++ b/install/installer/addNode_step2
@@ -379,11 +379,6 @@ sudo su $TRAF_USER --login --command "cd $TRAF_HOME/sql/scripts; sqregen -config
 
 }
 
-function rmsStart {
-
-sudo su $TRAF_USER --login --command "cd $SQ_ROOT/sql/scripts; rmsstart"
-}
-
 function dcsStart {
 
 sudo su $TRAF_USER --login --command "cd $TRAF_HOME/sql/scripts; dcsstart"
@@ -428,7 +423,6 @@ copySqcert
 addUpNodes
 sqRegen
 copysqConfig
-rmsStart
 dcsStart
 cleanUp
 printMessage

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/install/installer/traf_sqconfig
----------------------------------------------------------------------
diff --git a/install/installer/traf_sqconfig b/install/installer/traf_sqconfig
index 09fe04c..31315c2 100755
--- a/install/installer/traf_sqconfig
+++ b/install/installer/traf_sqconfig
@@ -78,10 +78,6 @@ done
 
 echo "end node" >> $sqconfig
 
-# The following sed is dependent on the nunber of lines in the copyright,
-# as they are excluded. If the copyright gets larger this needs to change.
-sed '1,20d' $LOCAL_WORKDIR/sqconfig_persist.txt >> $sqconfig
-
 sudo cp $sqconfig $UNTAR_DIR
 sudo cp $sqconfig $TRAF_USER_DIR/sqconfig
 sudo chown $TRAF_USER.$TRAF_GROUP $TRAF_USER_DIR/sqconfig

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/363df1f9/install/python-installer/scripts/traf_sqconfig.py
----------------------------------------------------------------------
diff --git a/install/python-installer/scripts/traf_sqconfig.py b/install/python-installer/scripts/traf_sqconfig.py
index 05c7a54..2895079 100755
--- a/install/python-installer/scripts/traf_sqconfig.py
+++ b/install/python-installer/scripts/traf_sqconfig.py
@@ -57,10 +57,6 @@ def run():
     with open(sqconfig_file, 'w') as f:
         f.writelines(lines)
 
-    # append the current persist section to the Trafodion configuration file
-    run_cmd('echo "sed \'1,20d\' %s >> %s"' % (sqconfig_persist_file, sqconfig_file))
-    run_cmd('sed \'1,20d\' %s >> %s' % (sqconfig_persist_file, sqconfig_file))
-
     print 'sqconfig generated successfully!'
 
     run_cmd('sqgen')