diff --git a/source/client/src/tmq.c b/source/client/src/tmq.c index bdd8c75f268f03fbf2dca5b9c5a1a23236a5c476..84de7f8de99edd89a78917b5660da7451d1effc0 100644 --- a/source/client/src/tmq.c +++ b/source/client/src/tmq.c @@ -961,7 +961,8 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { tmq_t* pTmq = taosMemoryCalloc(1, sizeof(tmq_t)); if (pTmq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - tscError("consumer %ld setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), pTmq->groupId); + tscError("consumer %" PRId64 " setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), + pTmq->groupId); return NULL; } @@ -979,7 +980,8 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { if (pTmq->clientTopics == NULL || pTmq->mqueue == NULL || pTmq->qall == NULL || pTmq->delayedTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - tscError("consumer %ld setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), pTmq->groupId); + tscError("consumer %" PRId64 " setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), + pTmq->groupId); goto FAIL; } @@ -1008,14 +1010,16 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { // init semaphore if (tsem_init(&pTmq->rspSem, 0, 0) != 0) { - tscError("consumer %ld setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), pTmq->groupId); + tscError("consumer %" PRId64 " setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), + pTmq->groupId); goto FAIL; } // init connection pTmq->pTscObj = taos_connect_internal(conf->ip, user, pass, NULL, NULL, conf->port, CONN_TYPE__TMQ); if (pTmq->pTscObj == NULL) { - tscError("consumer %ld setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), pTmq->groupId); + tscError("consumer %" PRId64 " setup failed since %s, consumer group %s", pTmq->consumerId, terrstr(), + pTmq->groupId); tsem_destroy(&pTmq->rspSem); goto FAIL; } @@ -1024,7 +1028,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->hbLiveTimer = taosTmrStart(tmqSendHbReq, 1000, pTmq, tmqMgmt.timer); } - tscInfo("consumer %ld is setup, consumer group %s", pTmq->consumerId, pTmq->groupId); + tscInfo("consumer %" PRId64 " is setup, consumer group %s", pTmq->consumerId, pTmq->groupId); return pTmq; diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 2a8f80f0308ccaf7f4b8d15d7351237fac9fb5e1..c5d82eabda9ba2da4dff33aea9cf115887160849 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -1773,8 +1773,10 @@ char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** pDataBuf) int32_t colNum = taosArrayGetSize(pDataBlock->pDataBlock); int32_t rows = pDataBlock->info.rows; int32_t len = 0; - len += snprintf(dumpBuf + len, size - len, "===stream===%s|block type %d|child id %d|group id:%" PRIu64 "|uid:%ld|rows:%d|version:%" PRIu64 "\n", flag, - (int32_t)pDataBlock->info.type, pDataBlock->info.childId, pDataBlock->info.groupId, + len += snprintf(dumpBuf + len, size - len, + "===stream===%s|block type %d|child id %d|group id:%" PRIu64 "|uid:%" PRId64 + "|rows:%d|version:%" PRIu64 "\n", + flag, (int32_t)pDataBlock->info.type, pDataBlock->info.childId, pDataBlock->info.groupId, pDataBlock->info.uid, pDataBlock->info.rows, pDataBlock->info.version); if (len >= size - 1) return dumpBuf; diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index f7387f7e887fe5beca3df87e6ca77d1d72f21632..39b57f29660091c8cd1a05daa75121b902f0646e 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -878,14 +878,14 @@ static int32_t mndRetrieveConsumer(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock * pShow->pIter = sdbFetch(pSdb, SDB_CONSUMER, pShow->pIter, (void **)&pConsumer); if (pShow->pIter == NULL) break; if (taosArrayGetSize(pConsumer->assignedTopics) == 0) { - mDebug("showing consumer %ld no assigned topic, skip", pConsumer->consumerId); + mDebug("showing consumer %" PRId64 " no assigned topic, skip", pConsumer->consumerId); sdbRelease(pSdb, pConsumer); continue; } taosRLockLatch(&pConsumer->lock); - mDebug("showing consumer %ld", pConsumer->consumerId); + mDebug("showing consumer %" PRId64, pConsumer->consumerId); int32_t topicSz = taosArrayGetSize(pConsumer->assignedTopics); bool hasTopic = true; diff --git a/source/dnode/vnode/src/meta/metaTable.c b/source/dnode/vnode/src/meta/metaTable.c index ff455bfe33e838d5bd308baba248dd295ea7c05a..fdf2da5558d36af12dda1a3143ab27550bc3d633 100644 --- a/source/dnode/vnode/src/meta/metaTable.c +++ b/source/dnode/vnode/src/meta/metaTable.c @@ -204,12 +204,12 @@ int metaCreateSTable(SMeta *pMeta, int64_t version, SVCreateStbReq *pReq) { ++pMeta->pVnode->config.vndStats.numOfSTables; - metaDebug("vgId:%d, super table is created, name:%s uid:%" PRId64, TD_VID(pMeta->pVnode), pReq->name, pReq->suid); + metaDebug("vgId:%d, stb:%s is created, suid:%" PRId64, TD_VID(pMeta->pVnode), pReq->name, pReq->suid); return 0; _err: - metaError("vgId:%d, failed to create super table:%s uid:%" PRId64 " since %s", TD_VID(pMeta->pVnode), pReq->name, + metaError("vgId:%d, failed to create stb:%s uid:%" PRId64 " since %s", TD_VID(pMeta->pVnode), pReq->name, pReq->suid, tstrerror(terrno)); return -1; } @@ -411,7 +411,7 @@ int metaCreateTable(SMeta *pMeta, int64_t version, SVCreateTbReq *pReq) { if (metaHandleEntry(pMeta, &me) < 0) goto _err; - metaDebug("vgId:%d, table %s uid %" PRId64 " is created, type:%" PRId8, TD_VID(pMeta->pVnode), pReq->name, pReq->uid, + metaDebug("vgId:%d, table:%s uid %" PRId64 " is created, type:%" PRId8, TD_VID(pMeta->pVnode), pReq->name, pReq->uid, pReq->type); return 0; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 32bfd1274e796490bb8bc0d9806b1897c6847b11..64dba7c992eb6b082c9e5b023132e5ce596d75ab 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -317,7 +317,7 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) { char buf[80]; tFormatOffset(buf, 80, &reqOffset); - tqDebug("tmq poll: consumer %ld (epoch %d), subkey %s, recv poll req in vg %d, req offset %s", consumerId, + tqDebug("tmq poll: consumer %" PRId64 " (epoch %d), subkey %s, recv poll req in vg %d, req offset %s", consumerId, pReq->epoch, pHandle->subKey, TD_VID(pTq->pVnode), buf); SMqDataRsp dataRsp = {0}; @@ -348,8 +348,8 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) { } } else if (reqOffset.type == TMQ_OFFSET__RESET_LATEST) { tqOffsetResetToLog(&dataRsp.rspOffset, walGetLastVer(pTq->pVnode->pWal)); - tqDebug("tmq poll: consumer %ld, subkey %s, vg %d, offset reset to %ld", consumerId, pHandle->subKey, - TD_VID(pTq->pVnode), dataRsp.rspOffset.version); + tqDebug("tmq poll: consumer %" PRId64 ", subkey %s, vg %d, offset reset to %" PRId64, consumerId, + pHandle->subKey, TD_VID(pTq->pVnode), dataRsp.rspOffset.version); if (tqSendDataRsp(pTq, pMsg, pReq, &dataRsp) < 0) { code = -1; } @@ -398,7 +398,7 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) { while (1) { consumerEpoch = atomic_load_32(&pHandle->epoch); if (consumerEpoch > reqEpoch) { - tqWarn("tmq poll: consumer %ld (epoch %d), subkey %s, vg %d offset %" PRId64 + tqWarn("tmq poll: consumer %" PRId64 " (epoch %d), subkey %s, vg %d offset %" PRId64 ", found new consumer epoch %d, discard req epoch %d", consumerId, pReq->epoch, pHandle->subKey, TD_VID(pTq->pVnode), fetchVer, consumerEpoch, reqEpoch); break; @@ -595,7 +595,7 @@ int32_t tqProcessVgChangeReq(STQ* pTq, char* msg, int32_t msgLen) { taosArrayDestroy(tbUidList); } taosHashPut(pTq->handles, req.subKey, strlen(req.subKey), pHandle, sizeof(STqHandle)); - tqDebug("try to persist handle %s consumer %ld", req.subKey, pHandle->consumerId); + tqDebug("try to persist handle %s consumer %" PRId64, req.subKey, pHandle->consumerId); if (tqMetaSaveHandle(pTq, req.subKey, pHandle) < 0) { // TODO ASSERT(0); @@ -714,7 +714,7 @@ int32_t tqProcessStreamTrigger(STQ* pTq, SSubmitReq* pReq, int64_t ver) { SStreamTask* pTask = *(SStreamTask**)pIter; if (!pTask->isDataScan) continue; - qDebug("data submit enqueue stream task: %d, ver: %ld", pTask->taskId, ver); + qDebug("data submit enqueue stream task: %d, ver: %" PRId64, pTask->taskId, ver); if (!failed) { if (streamTaskInput(pTask, (SStreamQueueItem*)pSubmit) < 0) { diff --git a/source/dnode/vnode/src/tq/tqExec.c b/source/dnode/vnode/src/tq/tqExec.c index 40dbbda60373551757c5c7e6b74dfb32e8d68d2b..435bbb77b8cab0b6c631f98e30444501ae8faf03 100644 --- a/source/dnode/vnode/src/tq/tqExec.c +++ b/source/dnode/vnode/src/tq/tqExec.c @@ -111,7 +111,8 @@ int64_t tqScan(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVa } if (pRsp->blockNum == 0 && pOffset->type == TMQ_OFFSET__SNAPSHOT_DATA) { - tqDebug("vgId: %d, tsdb consume over, switch to wal, ver %ld", TD_VID(pTq->pVnode), pHandle->snapshotVer + 1); + tqDebug("vgId: %d, tsdb consume over, switch to wal, ver %" PRId64, TD_VID(pTq->pVnode), + pHandle->snapshotVer + 1); tqOffsetResetToLog(pOffset, pHandle->snapshotVer); qStreamPrepareScan(task, pOffset); continue; diff --git a/source/dnode/vnode/src/tq/tqMeta.c b/source/dnode/vnode/src/tq/tqMeta.c index b8e021f795408f10b4363bf1199db82b90ebd2a6..5709ad7c85ff142bbe43cce6f5e70c6953d72459 100644 --- a/source/dnode/vnode/src/tq/tqMeta.c +++ b/source/dnode/vnode/src/tq/tqMeta.c @@ -92,7 +92,7 @@ int32_t tqMetaRestoreHandle(STQ* pTq) { handle.execHandle.execDb.pFilterOutTbUid = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK); } - tqDebug("tq restore %s consumer %ld vgId:%d", handle.subKey, handle.consumerId, TD_VID(pTq->pVnode)); + tqDebug("tq restore %s consumer %" PRId64 " vgId:%d", handle.subKey, handle.consumerId, TD_VID(pTq->pVnode)); taosHashPut(pTq->handles, pKey, kLen, &handle, sizeof(STqHandle)); } @@ -132,7 +132,7 @@ int32_t tqMetaSaveHandle(STQ* pTq, const char* key, const STqHandle* pHandle) { tEncodeSize(tEncodeSTqHandle, pHandle, vlen, code); ASSERT(code == 0); - tqDebug("tq save %s(%d) consumer %ld vgId:%d", pHandle->subKey, strlen(pHandle->subKey), pHandle->consumerId, + tqDebug("tq save %s(%d) consumer %" PRId64 " vgId:%d", pHandle->subKey, strlen(pHandle->subKey), pHandle->consumerId, TD_VID(pTq->pVnode)); void* buf = taosMemoryCalloc(1, vlen); diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 6f0b5af4f656fcc5aafc5297fb72f43cdbdccd67..f20c7e7e55f587ae72ce6864ade4ef25c5da0464 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -137,7 +137,7 @@ int32_t tqNextBlock(STqReader* pReader, SFetchRet* ret) { ret->offset.type = TMQ_OFFSET__LOG; ret->offset.version = pReader->ver; ret->fetchType = FETCH_TYPE__NONE; - tqDebug("return offset %ld, no more valid", ret->offset.version); + tqDebug("return offset %" PRId64 ", no more valid", ret->offset.version); ASSERT(ret->offset.version >= 0); return -1; } @@ -169,7 +169,7 @@ int32_t tqNextBlock(STqReader* pReader, SFetchRet* ret) { ret->offset.version = pReader->ver; ASSERT(pReader->ver >= 0); ret->fetchType = FETCH_TYPE__NONE; - tqDebug("return offset %ld, processed finish", ret->offset.version); + tqDebug("return offset %" PRId64 ", processed finish", ret->offset.version); return 0; } } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 1f3de502a9a4a1878a0cac2dbc6eb20d8e292487..6dbbf9962d2c76c2adf5479f322c7718a6a9b702 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -901,8 +901,7 @@ _exit: tdProcessRSmaSubmit(pVnode->pSma, pReq, STREAM_INPUT__DATA_SUBMIT); } - vDebug("successful submit in vg %d version %ld", pVnode->config.vgId, version); - + vDebug("vgId:%d, submit success, index:%" PRId64, pVnode->config.vgId, version); return 0; } diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 775017b8dd4c1257a92eec8c100a400d00e7f2f0..b2d4a5fc68df02fcffe2fd1547d123332cd787e0 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -605,7 +605,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, const STqOffsetVal* pOffset) { #if 0 if (tOffsetEqual(pOffset, &pTaskInfo->streamInfo.lastStatus) && pInfo->tqReader->pWalReader->curVersion != pOffset->version) { - qError("prepare scan ver %ld actual ver %ld, last %ld", pOffset->version, + qError("prepare scan ver %" PRId64 " actual ver %" PRId64 ", last %" PRId64, pOffset->version, pInfo->tqReader->pWalReader->curVersion, pTaskInfo->streamInfo.lastStatus.version); ASSERT(0); } @@ -636,8 +636,8 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, const STqOffsetVal* pOffset) { #ifndef NDEBUG - qDebug("switch to next table %ld (cursor %d), %ld rows returned", uid, pTableScanInfo->currentTable, - pInfo->pTableScanOp->resultInfo.totalRows); + qDebug("switch to next table %" PRId64 " (cursor %d), %" PRId64 " rows returned", uid, + pTableScanInfo->currentTable, pInfo->pTableScanOp->resultInfo.totalRows); pInfo->pTableScanOp->resultInfo.totalRows = 0; #endif @@ -669,8 +669,8 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, const STqOffsetVal* pOffset) { pTableScanInfo->cond.twindows.skey = oldSkey; pTableScanInfo->scanTimes = 0; - qDebug("tsdb reader offset seek to uid %ld ts %ld, table cur set to %d , all table num %d", uid, ts, - pTableScanInfo->currentTable, tableSz); + qDebug("tsdb reader offset seek to uid %" PRId64 " ts %" PRId64 ", table cur set to %d , all table num %d", uid, + ts, pTableScanInfo->currentTable, tableSz); /*}*/ } else { diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 6804a1258c2db16cf231042267e27de9b528116a..1e987ca49352e37dfb4050a31ab36b9bc90c766b 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -3995,7 +3995,7 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo int32_t sz = taosArrayGetSize(pTableListInfo->pTableList); for (int32_t i = 0; i < sz; i++) { STableKeyInfo* pKeyInfo = taosArrayGet(pTableListInfo->pTableList, i); - qDebug("creating stream task: add table %ld", pKeyInfo->uid); + qDebug("creating stream task: add table %" PRId64, pKeyInfo->uid); } } #endif diff --git a/source/libs/parser/src/parInsert.c b/source/libs/parser/src/parInsert.c index fbc5e1b4b0ebd010c10bd72d5417cfedd6041527..32e9ea835ba91e16d43237ec3e9d6f0e6de8f6a5 100644 --- a/source/libs/parser/src/parInsert.c +++ b/source/libs/parser/src/parInsert.c @@ -2408,9 +2408,9 @@ int32_t smlBindData(void* handle, SArray* tags, SArray* colsSchema, SArray* cols } else { int32_t colLen = kv->length; if (pColSchema->type == TSDB_DATA_TYPE_TIMESTAMP) { - // uError("SML:data before:%ld, precision:%d", kv->i, pTableMeta->tableInfo.precision); + // uError("SML:data before:%" PRId64 ", precision:%d", kv->i, pTableMeta->tableInfo.precision); kv->i = convertTimePrecision(kv->i, TSDB_TIME_PRECISION_NANO, pTableMeta->tableInfo.precision); - // uError("SML:data after:%ld, precision:%d", kv->i, pTableMeta->tableInfo.precision); + // uError("SML:data after:%" PRId64 ", precision:%d", kv->i, pTableMeta->tableInfo.precision); } if (IS_VAR_DATA_TYPE(kv->type)) { diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 31da865a69d8ab6a0efa0b1b305a107624e1f2d0..793a47d58ce7307e0e764a1a0c8514f522e49eca 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -143,7 +143,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, // enqueue if (pData != NULL) { - qDebug("task %d(child %d) recv retrieve req from task %d, reqId %ld", pTask->taskId, pTask->selfChildId, + qDebug("task %d(child %d) recv retrieve req from task %d, reqId %" PRId64, pTask->taskId, pTask->selfChildId, pReq->srcTaskId, pReq->reqId); pData->type = STREAM_INPUT__DATA_RETRIEVE; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 95e5b61dbddaf9b6fbfcac45badfa3c62d19da47..26cd9111bf7c3b9efee3a232755c841add2932df 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -159,8 +159,8 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) return -1; } - qDebug("task %d(child %d) send retrieve req to task %d at node %d, reqId %ld", pTask->taskId, pTask->selfChildId, - pEpInfo->taskId, pEpInfo->nodeId, req.reqId); + qDebug("task %d(child %d) send retrieve req to task %d at node %d, reqId %" PRId64, pTask->taskId, + pTask->selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); } return 0; FAIL: diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 196dbd6dc32e1b82c1b6cbd7a4409bbec90ff7d5..c5b8b20a8c87ad13147b4cf6492618f4fd3966c7 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -59,7 +59,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, void* data, SArray* pRes) block.info.childId = pTask->selfChildId; taosArrayPush(pRes, &block); - qDebug("task %d(child %d) processed retrieve, reqId %ld", pTask->taskId, pTask->selfChildId, + qDebug("task %d(child %d) processed retrieve, reqId %" PRId64, pTask->taskId, pTask->selfChildId, pRetrieveBlock->reqId); } break; diff --git a/source/libs/sync/src/syncAppendEntriesReply.c b/source/libs/sync/src/syncAppendEntriesReply.c index 7279f0858be7fa7c27406eb194b1b8e1f7c7d601..4928c54bd72acc079f2d4e4c42aac49d1ce33e82 100644 --- a/source/libs/sync/src/syncAppendEntriesReply.c +++ b/source/libs/sync/src/syncAppendEntriesReply.c @@ -93,7 +93,8 @@ int32_t syncNodeOnAppendEntriesReplyCb(SSyncNode* ths, SyncAppendEntriesReply* p SyncIndex afterMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId)); do { char logBuf[256]; - snprintf(logBuf, sizeof(logBuf), "before next:%ld, match:%ld, after next:%ld, match:%ld", beforeNextIndex, + snprintf(logBuf, sizeof(logBuf), + "before next:%" PRId64 ", match:%" PRId64 ", after next:%" PRId64 ", match:%" PRId64, beforeNextIndex, beforeMatchIndex, afterNextIndex, afterMatchIndex); syncLogRecvAppendEntriesReply(ths, pMsg, logBuf); } while (0); @@ -107,7 +108,7 @@ static void syncNodeStartSnapshotOnce(SSyncNode* ths, SyncIndex beginIndex, Sync if (beginIndex > endIndex) { do { char logBuf[128]; - snprintf(logBuf, sizeof(logBuf), "snapshot param error, start:%ld, end:%ld", beginIndex, endIndex); + snprintf(logBuf, sizeof(logBuf), "snapshot param error, start:%" PRId64 ", end:%" PRId64, beginIndex, endIndex); syncNodeErrorLog(ths, logBuf); } while (0); @@ -293,7 +294,8 @@ int32_t syncNodeOnAppendEntriesReplySnapshot2Cb(SSyncNode* ths, SyncAppendEntrie SyncIndex afterMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId)); do { char logBuf[256]; - snprintf(logBuf, sizeof(logBuf), "before next:%ld, match:%ld, after next:%ld, match:%ld", beforeNextIndex, + snprintf(logBuf, sizeof(logBuf), + "before next:%" PRId64 ", match:%" PRId64 ", after next:%" PRId64 ", match:%" PRId64, beforeNextIndex, beforeMatchIndex, afterNextIndex, afterMatchIndex); syncLogRecvAppendEntriesReply(ths, pMsg, logBuf); } while (0); @@ -392,7 +394,8 @@ int32_t syncNodeOnAppendEntriesReplySnapshotCb(SSyncNode* ths, SyncAppendEntries SyncIndex afterMatchIndex = syncIndexMgrGetIndex(ths->pMatchIndex, &(pMsg->srcId)); do { char logBuf[256]; - snprintf(logBuf, sizeof(logBuf), "before next:%ld, match:%ld, after next:%ld, match:%ld", beforeNextIndex, + snprintf(logBuf, sizeof(logBuf), + "before next:%" PRId64 ", match:%" PRId64 ", after next:%" PRId64 ", match:%" PRId64, beforeNextIndex, beforeMatchIndex, afterNextIndex, afterMatchIndex); syncLogRecvAppendEntriesReply(ths, pMsg, logBuf); } while (0); diff --git a/source/libs/sync/src/syncCommit.c b/source/libs/sync/src/syncCommit.c index fd6577477f171402f1769a02e18b60b765339e04..c18c2cc0d596082b256a2a0ee5a670d347068cf6 100644 --- a/source/libs/sync/src/syncCommit.c +++ b/source/libs/sync/src/syncCommit.c @@ -82,8 +82,8 @@ void syncMaybeAdvanceCommitIndex(SSyncNode* pSyncNode) { } else { do { char logBuf[128]; - snprintf(logBuf, sizeof(logBuf), "can not commit due to term not equal, index:%ld, term:%lu", pEntry->index, - pEntry->term); + snprintf(logBuf, sizeof(logBuf), "can not commit due to term not equal, index:%" PRId64 ", term:%" PRIu64, + pEntry->index, pEntry->term); syncNodeEventLog(pSyncNode, logBuf); } while (0); } diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index 1b802b9e5b4f613f56fdf452925ccc4bd894d209..10b14ffcbd8f90b7dc9b671b9b2b0f815557430f 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -2238,7 +2238,8 @@ SyncTerm syncNodeGetPreTerm(SSyncNode* pSyncNode, SyncIndex index) { do { char logBuf[128]; - snprintf(logBuf, sizeof(logBuf), "sync node get pre term error, index:%ld, snap-index:%ld, snap-term:%lu", index, + snprintf(logBuf, sizeof(logBuf), + "sync node get pre term error, index:%" PRId64 ", snap-index:%" PRId64 ", snap-term:%" PRIu64, index, snapshot.lastApplyIndex, snapshot.lastApplyTerm); syncNodeErrorLog(pSyncNode, logBuf); } while (0); diff --git a/source/libs/sync/test/syncEntryCacheTest.cpp b/source/libs/sync/test/syncEntryCacheTest.cpp index fd0c41cce9c8ed1d37abe000e5404926f7729fc1..3ee28ce96b00e0722402af586bdad30269af8e31 100644 --- a/source/libs/sync/test/syncEntryCacheTest.cpp +++ b/source/libs/sync/test/syncEntryCacheTest.cpp @@ -82,12 +82,12 @@ void test2() { code = raftEntryCacheGetEntryP(pCache, index, &pEntry); ASSERT(code == 1 && index == pEntry->index); - sTrace("get entry:%p for %ld", pEntry, index); + sTrace("get entry:%p for %" PRId64, pEntry, index); syncEntryLog2((char*)"==test2 get entry pointer 2==", pEntry); code = raftEntryCacheGetEntry(pCache, index, &pEntry); ASSERT(code == 1 && index == pEntry->index); - sTrace("get entry:%p for %ld", pEntry, index); + sTrace("get entry:%p for %" PRId64, pEntry, index); syncEntryLog2((char*)"==test2 get entry 2==", pEntry); syncEntryDestory(pEntry); @@ -95,14 +95,14 @@ void test2() { index = 8; code = raftEntryCacheGetEntry(pCache, index, &pEntry); ASSERT(code == 0); - sTrace("get entry:%p for %ld", pEntry, index); + sTrace("get entry:%p for %" PRId64, pEntry, index); sTrace("==test2 get entry 8 not found=="); // not found index = 9; code = raftEntryCacheGetEntry(pCache, index, &pEntry); ASSERT(code == 0); - sTrace("get entry:%p for %ld", pEntry, index); + sTrace("get entry:%p for %" PRId64, pEntry, index); sTrace("==test2 get entry 9 not found=="); } @@ -135,7 +135,7 @@ void test4() { ASSERT(pEntry != NULL); int64_t rid = taosAddRef(testRefId, pEntry); - sTrace("rid: %ld", rid); + sTrace("rid: %" PRId64, rid); do { SSyncRaftEntry* pAcquireEntry = (SSyncRaftEntry*)taosAcquireRef(testRefId, rid); @@ -164,7 +164,7 @@ void test5() { ASSERT(pEntry != NULL); int64_t rid = taosAddRef(testRefId, pEntry); - sTrace("rid: %ld", rid); + sTrace("rid: %" PRId64, rid); } for (int64_t rid = 2; rid < 101; rid++) { diff --git a/source/libs/sync/test/syncHashCacheTest.cpp b/source/libs/sync/test/syncHashCacheTest.cpp index f155bd834fb1ac860c72372b1a99ad7a3f0b1474..7d822971da61462a7f9f65321bb2e72103fa3c27 100644 --- a/source/libs/sync/test/syncHashCacheTest.cpp +++ b/source/libs/sync/test/syncHashCacheTest.cpp @@ -194,13 +194,13 @@ SSyncRaftEntry* getLogEntry2(SSkipList* pSkipList, SyncIndex index) { } taosArrayDestroy(entryPArray); - sTrace("get index2: %ld, arraySize:%d -------------", index, arraySize); + sTrace("get index2: %" PRId64 ", arraySize:%d -------------", index, arraySize); syncEntryLog2((char*)"getLogEntry2", pEntry); return pEntry; } SSyncRaftEntry* getLogEntry(SSkipList* pSkipList, SyncIndex index) { - sTrace("get index: %ld -------------", index); + sTrace("get index: %" PRId64 " -------------", index); SyncIndex index2 = index; SSyncRaftEntry* pEntry = NULL; SSkipListIterator* pIter = diff --git a/source/libs/wal/src/walRead.c b/source/libs/wal/src/walRead.c index f57bcd41d6a9422b5d5f1ad0c3c9cc16c6770e81..a5b5a2b7b4cac113978d8278ecf0a57686a67257 100644 --- a/source/libs/wal/src/walRead.c +++ b/source/libs/wal/src/walRead.c @@ -78,7 +78,8 @@ int32_t walNextValidMsg(SWalReader *pReader) { int64_t endVer = pReader->cond.scanUncommited ? lastVer : committedVer; endVer = TMIN(appliedVer, endVer); - wDebug("vgId:%d, wal start to fetch, ver %ld, last ver %ld commit ver %ld, applied ver %ld, end ver %ld", + wDebug("vgId:%d, wal start to fetch, index:%" PRId64 ", last index:%" PRId64 " commit index:%" PRId64 + ", applied index:%" PRId64 ", end index:%" PRId64, pReader->pWal->cfg.vgId, fetchVer, lastVer, committedVer, appliedVer, endVer); pReader->curStopped = 0; while (fetchVer <= endVer) { @@ -190,7 +191,7 @@ int32_t walReadSeekVerImpl(SWalReader *pReader, int64_t ver) { return -1; } - wDebug("vgId:%d, wal version reset from %" PRId64 "(invalid: %d) to %" PRId64, pReader->pWal->cfg.vgId, + wDebug("vgId:%d, wal version reset from index:%" PRId64 "(invalid:%d) to index:%" PRId64, pReader->pWal->cfg.vgId, pReader->curVersion, pReader->curInvalid, ver); pReader->curVersion = ver; @@ -200,7 +201,7 @@ int32_t walReadSeekVerImpl(SWalReader *pReader, int64_t ver) { int32_t walReadSeekVer(SWalReader *pReader, int64_t ver) { SWal *pWal = pReader->pWal; if (!pReader->curInvalid && ver == pReader->curVersion) { - wDebug("vgId:%d, wal version %" PRId64 " match, no need to reset", pReader->pWal->cfg.vgId, ver); + wDebug("vgId:%d, wal index:%" PRId64 " match, no need to reset", pReader->pWal->cfg.vgId, ver); return 0; } @@ -229,7 +230,7 @@ static int32_t walFetchHeadNew(SWalReader *pRead, int64_t fetchVer) { int64_t contLen; bool seeked = false; - wDebug("vgId:%d, wal starts to fetch head %d", pRead->pWal->cfg.vgId, fetchVer); + wDebug("vgId:%d, wal starts to fetch head, index:%" PRId64, pRead->pWal->cfg.vgId, fetchVer); if (pRead->curInvalid || pRead->curVersion != fetchVer) { if (walReadSeekVer(pRead, fetchVer) < 0) { @@ -267,7 +268,7 @@ static int32_t walFetchBodyNew(SWalReader *pRead) { SWalCont *pReadHead = &pRead->pHead->head; int64_t ver = pReadHead->version; - wDebug("vgId:%d, wal starts to fetch body %ld", pRead->pWal->cfg.vgId, ver); + wDebug("vgId:%d, wal starts to fetch body, index:%" PRId64, pRead->pWal->cfg.vgId, ver); if (pRead->capacity < pReadHead->bodyLen) { void *ptr = taosMemoryRealloc(pRead->pHead, sizeof(SWalCkHead) + pReadHead->bodyLen); @@ -312,7 +313,7 @@ static int32_t walFetchBodyNew(SWalReader *pRead) { return -1; } - wDebug("vgId:%d, version %" PRId64 " is fetched, cursor advance", pRead->pWal->cfg.vgId, ver); + wDebug("vgId:%d, index:%" PRId64 " is fetched, cursor advance", pRead->pWal->cfg.vgId, ver); pRead->curVersion = ver + 1; return 0; } @@ -406,7 +407,7 @@ int32_t walFetchBody(SWalReader *pRead, SWalCkHead **ppHead) { } if (pReadHead->version != ver) { - wError("vgId:%d, wal fetch body error:%" PRId64 ", read request index:%" PRId64, pRead->pWal->cfg.vgId, + wError("vgId:%d, wal fetch body error, index:%" PRId64 ", read request index:%" PRId64, pRead->pWal->cfg.vgId, pRead->pHead->head.version, ver); pRead->curInvalid = 1; terrno = TSDB_CODE_WAL_FILE_CORRUPTED; @@ -414,7 +415,7 @@ int32_t walFetchBody(SWalReader *pRead, SWalCkHead **ppHead) { } if (walValidBodyCksum(*ppHead) != 0) { - wError("vgId:%d, wal fetch body error:%" PRId64 ", since body checksum not passed", pRead->pWal->cfg.vgId, ver); + wError("vgId:%d, wal fetch body error, index:%" PRId64 ", since body checksum not passed", pRead->pWal->cfg.vgId, ver); pRead->curInvalid = 1; terrno = TSDB_CODE_WAL_FILE_CORRUPTED; return -1; @@ -425,7 +426,7 @@ int32_t walFetchBody(SWalReader *pRead, SWalCkHead **ppHead) { } int32_t walReadVer(SWalReader *pReader, int64_t ver) { - wDebug("vgId:%d, wal start to read ver %ld", pReader->pWal->cfg.vgId, ver); + wDebug("vgId:%d, wal start to read index:%" PRId64, pReader->pWal->cfg.vgId, ver); int64_t contLen; int32_t code; bool seeked = false; @@ -521,7 +522,7 @@ int32_t walReadVer(SWalReader *pReader, int64_t ver) { ver); uint32_t readCkSum = walCalcBodyCksum(pReader->pHead->head.body, pReader->pHead->head.bodyLen); uint32_t logCkSum = pReader->pHead->cksumBody; - wError("checksum written into log: %u, checksum calculated: %u", logCkSum, readCkSum); + wError("checksum written into log:%u, checksum calculated:%u", logCkSum, readCkSum); pReader->curInvalid = 1; terrno = TSDB_CODE_WAL_FILE_CORRUPTED; ASSERT(0);