scheduler.c 30.0 KB
Newer Older
H
refact  
Hongze Cheng 已提交
1 2 3 4 5 6 7 8 9 10 11 12 13
/*
 * Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
 *
 * This program is free software: you can use, redistribute, and/or modify
 * it under the terms of the GNU Affero General Public License, version 3
 * or later ("AGPL"), as published by the Free Software Foundation.
 *
 * 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.
 *
 * You should have received a copy of the GNU Affero General Public License
 * along with this program. If not, see <http://www.gnu.org/licenses/>.
14 15
 */

D
dapan1121 已提交
16
#include "schedulerInt.h"
H
Hongze Cheng 已提交
17
#include "tmsg.h"
18
#include "query.h"
D
dapan 已提交
19
#include "catalog.h"
20 21

SSchedulerMgmt schMgmt = {0};
D
dapan1121 已提交
22 23


D
dapan 已提交
24
int32_t schBuildTaskRalation(SSchJob *job, SHashObj *planToTask) {
D
dapan1121 已提交
25
  for (int32_t i = 0; i < job->levelNum; ++i) {
D
dapan 已提交
26
    SSchLevel *level = taosArrayGet(job->levels, i);
D
dapan 已提交
27
    
D
dapan1121 已提交
28
    for (int32_t m = 0; m < level->taskNum; ++m) {
D
dapan 已提交
29
      SSchTask *task = taosArrayGet(level->subTasks, m);
D
dapan1121 已提交
30
      SSubplan *plan = task->plan;
H
Haojun Liao 已提交
31
      int32_t childNum = plan->pChildren ? (int32_t)taosArrayGetSize(plan->pChildren) : 0;
D
dapan1121 已提交
32
      int32_t parentNum = plan->pParents ? (int32_t)taosArrayGetSize(plan->pParents) : 0;
D
dapan1121 已提交
33 34

      if (childNum > 0) {
D
dapan 已提交
35 36
        task->children = taosArrayInit(childNum, POINTER_BYTES);
        if (NULL == task->children) {
D
dapan1121 已提交
37 38 39 40 41 42
          qError("taosArrayInit %d failed", childNum);
          SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
        }
      }

      for (int32_t n = 0; n < childNum; ++n) {
H
Haojun Liao 已提交
43
        SSubplan **child = taosArrayGet(plan->pChildren, n);
D
dapan 已提交
44
        SSchTask **childTask = taosHashGet(planToTask, child, POINTER_BYTES);
D
dapan 已提交
45
        if (NULL == childTask || NULL == *childTask) {
D
dapan1121 已提交
46 47 48 49
          qError("subplan relationship error, level:%d, taskIdx:%d, childIdx:%d", i, m, n);
          SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
        }

D
dapan 已提交
50
        if (NULL == taosArrayPush(task->children, childTask)) {
D
dapan1121 已提交
51 52 53 54 55 56 57 58 59 60 61 62 63 64
          qError("taosArrayPush failed");
          SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
        }
      }

      if (parentNum > 0) {
        task->parents = taosArrayInit(parentNum, POINTER_BYTES);
        if (NULL == task->parents) {
          qError("taosArrayInit %d failed", parentNum);
          SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
        }
      }

      for (int32_t n = 0; n < parentNum; ++n) {
D
dapan 已提交
65
        SSubplan **parent = taosArrayGet(plan->pParents, n);
D
dapan 已提交
66
        SSchTask **parentTask = taosHashGet(planToTask, parent, POINTER_BYTES);
D
dapan 已提交
67
        if (NULL == parentTask || NULL == *parentTask) {
D
dapan1121 已提交
68 69 70 71
          qError("subplan relationship error, level:%d, taskIdx:%d, childIdx:%d", i, m, n);
          SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
        }

D
dapan 已提交
72
        if (NULL == taosArrayPush(task->parents, parentTask)) {
D
dapan1121 已提交
73 74 75 76 77 78 79
          qError("taosArrayPush failed");
          SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
        }
      }      
    }
  }

D
dapan 已提交
80 81
  SSchLevel *level = taosArrayGet(job->levels, 0);
  if (job->attr.queryJob && level->taskNum > 1) {
D
dapan 已提交
82 83 84 85
    qError("invalid plan info, level 0, taskNum:%d", level->taskNum);
    SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
  }

D
dapan 已提交
86
  SSchTask *task = taosArrayGet(level->subTasks, 0);
D
dapan 已提交
87 88 89 90 91 92
  if (task->parents && taosArrayGetSize(task->parents) > 0) {
    qError("invalid plan info, level 0, parentNum:%d", (int32_t)taosArrayGetSize(task->parents));
    SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
  }


D
dapan1121 已提交
93 94 95 96
  return TSDB_CODE_SUCCESS;
}


D
dapan 已提交
97
int32_t schValidateAndBuildJob(SQueryDag *dag, SSchJob *job) {
D
dapan1121 已提交
98
  int32_t code = 0;
D
dapan 已提交
99

D
dapan1121 已提交
100 101
  job->queryId = dag->queryId;
  
D
dapan 已提交
102 103 104 105 106
  if (dag->numOfSubplans <= 0) {
    qError("invalid subplan num:%d", dag->numOfSubplans);
    SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT);
  }
  
107 108 109
  int32_t levelNum = (int32_t)taosArrayGetSize(dag->pSubplans);
  if (levelNum <= 0) {
    qError("invalid level num:%d", levelNum);
D
dapan1121 已提交
110
    SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT);
111 112
  }

D
dapan1121 已提交
113 114 115 116 117 118
  SHashObj *planToTask = taosHashInit(SCHEDULE_DEFAULT_TASK_NUMBER, taosGetDefaultHashFunction(POINTER_BYTES == sizeof(int64_t) ? TSDB_DATA_TYPE_BIGINT : TSDB_DATA_TYPE_INT), false, HASH_NO_LOCK);
  if (NULL == planToTask) {
    qError("taosHashInit %d failed", SCHEDULE_DEFAULT_TASK_NUMBER);
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }
  
D
dapan 已提交
119
  job->levels = taosArrayInit(levelNum, sizeof(SSchLevel));
120 121
  if (NULL == job->levels) {
    qError("taosArrayInit %d failed", levelNum);
D
dapan1121 已提交
122
    SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
123 124
  }

H
Haojun Liao 已提交
125
  //??
D
dapan1121 已提交
126 127
  job->attr.needFetch = true;
  
128 129 130 131 132
  job->levelNum = levelNum;
  job->levelIdx = levelNum - 1;

  job->subPlans = dag->pSubplans;

D
dapan 已提交
133
  SSchLevel level = {0};
134 135
  SArray *levelPlans = NULL;
  int32_t levelPlanNum = 0;
D
dapan 已提交
136
  SSchLevel *pLevel = NULL;
137

D
dapan1121 已提交
138
  level.status = JOB_TASK_STATUS_NOT_START;
D
dapan1121 已提交
139

140
  for (int32_t i = 0; i < levelNum; ++i) {
D
dapan1121 已提交
141 142 143 144 145 146 147 148
    if (NULL == taosArrayPush(job->levels, &level)) {
      qError("taosArrayPush failed");
      SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
    }

    pLevel = taosArrayGet(job->levels, i);
  
    pLevel->level = i;
149 150 151
    levelPlans = taosArrayGetP(dag->pSubplans, i);
    if (NULL == levelPlans) {
      qError("no level plans for level %d", i);
D
dapan1121 已提交
152
      SCH_ERR_JRET(TSDB_CODE_QRY_INVALID_INPUT);
153 154 155 156 157
    }

    levelPlanNum = (int32_t)taosArrayGetSize(levelPlans);
    if (levelPlanNum <= 0) {
      qError("invalid level plans number:%d, level:%d", levelPlanNum, i);
D
dapan1121 已提交
158
      SCH_ERR_JRET(TSDB_CODE_QRY_INVALID_INPUT);
D
dapan1121 已提交
159 160
    }

D
dapan1121 已提交
161
    pLevel->taskNum = levelPlanNum;
D
dapan1121 已提交
162
    
D
dapan 已提交
163
    pLevel->subTasks = taosArrayInit(levelPlanNum, sizeof(SSchTask));
D
dapan1121 已提交
164
    if (NULL == pLevel->subTasks) {
D
dapan1121 已提交
165
      qError("taosArrayInit %d failed", levelPlanNum);
D
dapan1121 已提交
166
      SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
167 168 169
    }
    
    for (int32_t n = 0; n < levelPlanNum; ++n) {
X
Xiaoyu Wang 已提交
170
      SSubplan *plan = taosArrayGetP(levelPlans, n);
D
dapan 已提交
171
      SSchTask task = {0};
D
dapan1121 已提交
172 173 174

      if (plan->type == QUERY_TYPE_MODIFY) {
        job->attr.needFetch = false;
D
dapan 已提交
175 176
      } else {
        job->attr.queryJob = true;
D
dapan1121 已提交
177
      }
D
dapan 已提交
178

D
dapan1121 已提交
179
      
D
dapan1121 已提交
180 181
      task.taskId = atomic_add_fetch_64(&schMgmt.taskId, 1);
      task.plan = plan;
D
dapan1121 已提交
182
      task.level = pLevel;
183
      task.status = JOB_TASK_STATUS_NOT_START;
D
dapan1121 已提交
184

D
dapan1121 已提交
185
      void *p = taosArrayPush(pLevel->subTasks, &task);
D
dapan1121 已提交
186 187 188 189 190 191
      if (NULL == p) {
        qError("taosArrayPush failed");
        SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
      }
      
      if (0 != taosHashPut(planToTask, &plan, POINTER_BYTES, &p, POINTER_BYTES)) {
D
dapan1121 已提交
192 193 194
        qError("taosHashPut failed");
        SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
      }
D
dapan1121 已提交
195
    }
196 197 198

  }

D
dapan1121 已提交
199 200 201 202 203 204
  SCH_ERR_JRET(schBuildTaskRalation(job, planToTask));

  if (planToTask) {
    taosHashCleanup(planToTask);
  }
  
205
  return TSDB_CODE_SUCCESS;
D
dapan1121 已提交
206 207

_return:
D
dapan1121 已提交
208 209
  if (pLevel->subTasks) {
    taosArrayDestroy(pLevel->subTasks);
D
dapan1121 已提交
210 211
  }

D
dapan1121 已提交
212 213 214 215
  if (planToTask) {
    taosHashCleanup(planToTask);
  }

D
dapan1121 已提交
216
  SCH_RET(code);
217 218
}

D
dapan1121 已提交
219 220
int32_t schSetTaskCondidateAddrs(SSchJob *job, SSchTask *task) {
  if (task->condidateAddrs) {
D
dapan 已提交
221 222 223
    return TSDB_CODE_SUCCESS;
  }

D
dapan1121 已提交
224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240
  task->condidateIdx = 0;
  task->condidateAddrs = taosArrayInit(SCH_MAX_CONDIDATE_EP_NUM, sizeof(SQueryNodeAddr));
  if (NULL == task->condidateAddrs) {
    qError("taosArrayInit failed");
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  if (task->plan->execNode.numOfEps > 0) {
    if (NULL == taosArrayPush(task->condidateAddrs, &task->plan->execNode)) {
      qError("taosArrayPush failed");
      SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
    }

    return TSDB_CODE_SUCCESS;
  }

  int32_t addNum = 0;
D
dapan1121 已提交
241
  int32_t nodeNum = taosArrayGetSize(job->nodeList);
D
dapan1121 已提交
242
  
D
dapan1121 已提交
243 244
  for (int32_t i = 0; i < nodeNum && addNum < SCH_MAX_CONDIDATE_EP_NUM; ++i) {
    SQueryNodeAddr *naddr = taosArrayGet(job->nodeList, i);
D
dapan1121 已提交
245
    
D
dapan1121 已提交
246 247 248 249
    if (NULL == taosArrayPush(task->condidateAddrs, &task->plan->execNode)) {
      qError("taosArrayPush failed");
      SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
    }
D
dapan1121 已提交
250
    
D
dapan1121 已提交
251
    ++addNum;
D
dapan1121 已提交
252 253
  }

D
dapan1121 已提交
254 255
/*
  for (int32_t i = 0; i < job->dataSrcEps.numOfEps && addNum < SCH_MAX_CONDIDATE_EP_NUM; ++i) {
D
dapan1121 已提交
256 257 258 259
    strncpy(epSet->fqdn[epSet->numOfEps], job->dataSrcEps.fqdn[i], sizeof(job->dataSrcEps.fqdn[i]));
    epSet->port[epSet->numOfEps] = job->dataSrcEps.port[i];
    
    ++epSet->numOfEps;
D
dapan 已提交
260
  }
D
dapan1121 已提交
261
*/
D
dapan 已提交
262 263

  return TSDB_CODE_SUCCESS;
D
dapan1121 已提交
264
}
D
dapan1121 已提交
265

D
dapan1121 已提交
266

D
dapan 已提交
267
int32_t schPushTaskToExecList(SSchJob *job, SSchTask *task) {
D
dapan 已提交
268 269 270 271 272 273 274 275
  if (0 != taosHashPut(job->execTasks, &task->taskId, sizeof(task->taskId), &task, POINTER_BYTES)) {
    qError("taosHashPut failed");
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  return TSDB_CODE_SUCCESS;
}

D
dapan 已提交
276
int32_t schMoveTaskToSuccList(SSchJob *job, SSchTask *task, bool *moved) {
D
dapan 已提交
277 278 279 280 281
  if (0 != taosHashRemove(job->execTasks, &task->taskId, sizeof(task->taskId))) {
    qWarn("remove task[%"PRIx64"] from execTasks failed", task->taskId);
    return TSDB_CODE_SUCCESS;
  }

D
dapan1121 已提交
282 283 284 285 286 287 288 289 290 291
  if (0 != taosHashPut(job->succTasks, &task->taskId, sizeof(task->taskId), &task, POINTER_BYTES)) {
    qError("taosHashPut failed");
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  *moved = true;
  
  return TSDB_CODE_SUCCESS;
}

D
dapan 已提交
292
int32_t schMoveTaskToFailList(SSchJob *job, SSchTask *task, bool *moved) {
D
dapan1121 已提交
293
  if (0 != taosHashRemove(job->execTasks, &task->taskId, sizeof(task->taskId))) {
D
dapan 已提交
294
    qWarn("remove task[%"PRIx64"] from execTasks failed, it may not exist", task->taskId);
D
dapan1121 已提交
295 296 297
  }

  if (0 != taosHashPut(job->failTasks, &task->taskId, sizeof(task->taskId), &task, POINTER_BYTES)) {
D
dapan 已提交
298 299 300 301 302 303 304 305 306
    qError("taosHashPut failed");
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  *moved = true;
  
  return TSDB_CODE_SUCCESS;
}

D
dapan 已提交
307
int32_t schTaskCheckAndSetRetry(SSchJob *job, SSchTask *task, int32_t errCode, bool *needRetry) {
D
dapan1121 已提交
308 309
  // TODO set retry or not based on task type/errCode/retry times/job status/available eps...
  // TODO if needRetry, set task retry info
D
dapan 已提交
310

D
dapan1121 已提交
311 312 313
  *needRetry = false;

  return TSDB_CODE_SUCCESS;
D
dapan 已提交
314 315
}

D
dapan 已提交
316

D
dapan 已提交
317
int32_t schFetchFromRemote(SSchJob *job) {
D
dapan 已提交
318 319 320 321 322 323 324
  int32_t code = 0;
  
  if (atomic_val_compare_exchange_32(&job->remoteFetch, 0, 1) != 0) {
    qInfo("prior fetching not finished");
    return TSDB_CODE_SUCCESS;
  }

D
dapan1121 已提交
325
  SCH_ERR_JRET(schBuildAndSendMsg(job, job->fetchTask, TDMT_VND_FETCH));
D
dapan 已提交
326 327 328 329 330

  return TSDB_CODE_SUCCESS;
  
_return:
  atomic_val_compare_exchange_32(&job->remoteFetch, 1, 0);
D
dapan 已提交
331

D
dapan 已提交
332
  return code;
D
dapan 已提交
333 334
}

D
dapan 已提交
335

D
dapan 已提交
336
int32_t schProcessOnJobPartialSuccess(SSchJob *job) {
D
dapan1121 已提交
337 338
  job->status = JOB_TASK_STATUS_PARTIAL_SUCCEED;

D
dapan 已提交
339
  if ((!job->attr.needFetch) && job->attr.syncSchedule) {
D
dapan1121 已提交
340 341
    tsem_post(&job->rspSem);
  }
D
dapan1121 已提交
342
  
D
dapan 已提交
343 344 345
  if (job->userFetch) {
    SCH_ERR_RET(schFetchFromRemote(job));
  }
D
dapan 已提交
346

D
dapan 已提交
347
  return TSDB_CODE_SUCCESS;
D
dapan 已提交
348 349
}

D
dapan 已提交
350
int32_t schProcessOnJobFailure(SSchJob *job, int32_t errCode) {
D
dapan1121 已提交
351
  job->status = JOB_TASK_STATUS_FAILED;
D
dapan 已提交
352
  job->errCode = errCode;
D
dapan1121 已提交
353

D
dapan 已提交
354 355
  atomic_val_compare_exchange_32(&job->remoteFetch, 1, 0);

D
dapan 已提交
356
  if (job->userFetch || ((!job->attr.needFetch) && job->attr.syncSchedule)) {
D
dapan 已提交
357 358
    tsem_post(&job->rspSem);
  }
D
dapan 已提交
359

D
dapan 已提交
360 361 362
  return TSDB_CODE_SUCCESS;
}

D
dapan 已提交
363
int32_t schProcessOnDataFetched(SSchJob *job) {
D
dapan 已提交
364 365 366
  atomic_val_compare_exchange_32(&job->remoteFetch, 1, 0);

  tsem_post(&job->rspSem);
D
dapan 已提交
367 368 369
}


D
dapan 已提交
370
int32_t schProcessOnTaskSuccess(SSchJob *job, SSchTask *task) {
D
dapan 已提交
371 372 373 374
  bool moved = false;
  
  SCH_ERR_RET(schMoveTaskToSuccList(job, task, &moved));
  if (!moved) {
D
dapan 已提交
375
    SCH_TASK_ERR_LOG("task may already moved, status:%d", task->status);
D
dapan 已提交
376 377
    return TSDB_CODE_SUCCESS;
  }
D
dapan1121 已提交
378

D
dapan1121 已提交
379
  task->status = JOB_TASK_STATUS_SUCCEED;
D
dapan 已提交
380
  
D
dapan 已提交
381
  int32_t parentNum = task->parents ? (int32_t)taosArrayGetSize(task->parents) : 0;
D
dapan 已提交
382 383 384 385 386 387
  if (parentNum == 0) {
    if (task->plan->level != 0) {
      qError("level error");
      SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT);
    }

D
dapan1121 已提交
388 389 390 391
    int32_t taskDone = 0;
    
    if (SCH_TASK_NEED_WAIT_ALL(task)) {
      SCH_LOCK(SCH_WRITE, &task->level->lock);
D
dapan 已提交
392
      task->level->taskSucceed++;
D
dapan1121 已提交
393 394 395 396 397 398 399 400 401 402
      taskDone = task->level->taskSucceed + task->level->taskFailed;
      SCH_UNLOCK(SCH_WRITE, &task->level->lock);
      
      if (taskDone < task->level->taskNum) {
        qDebug("wait all tasks, done:%d, all:%d", taskDone, task->level->taskNum);
        return TSDB_CODE_SUCCESS;
      }

      if (task->level->taskFailed > 0) {
        job->status = JOB_TASK_STATUS_FAILED;
D
dapan 已提交
403
        SCH_ERR_RET(schProcessOnJobFailure(job, TSDB_CODE_QRY_APP_ERROR));
D
dapan1121 已提交
404 405 406 407

        return TSDB_CODE_SUCCESS;
      }
    } else {
D
dapan1121 已提交
408 409
      strncpy(job->resEp.fqdn, task->execAddr.epAddr[task->execAddr.inUse].fqdn, sizeof(job->resEp.fqdn));
      job->resEp.port = task->execAddr.epAddr[task->execAddr.inUse].port;
D
dapan1121 已提交
410
    }
D
dapan 已提交
411

D
dapan 已提交
412 413
    job->fetchTask = task;
    
D
dapan1121 已提交
414
    SCH_ERR_RET(schProcessOnJobPartialSuccess(job));
D
dapan 已提交
415 416 417 418

    return TSDB_CODE_SUCCESS;
  }

D
dapan1121 已提交
419
/*
D
dapan 已提交
420
  if (SCH_IS_DATA_SRC_TASK(task) && job->dataSrcEps.numOfEps < SCH_MAX_CONDIDATE_EP_NUM) {
D
dapan 已提交
421 422 423 424 425
    strncpy(job->dataSrcEps.fqdn[job->dataSrcEps.numOfEps], task->execAddr.fqdn, sizeof(task->execAddr.fqdn));
    job->dataSrcEps.port[job->dataSrcEps.numOfEps] = task->execAddr.port;

    ++job->dataSrcEps.numOfEps;
  }
D
dapan1121 已提交
426
*/
D
dapan 已提交
427

D
dapan 已提交
428
  for (int32_t i = 0; i < parentNum; ++i) {
D
dapan 已提交
429
    SSchTask *par = *(SSchTask **)taosArrayGet(task->parents, i);
D
dapan 已提交
430 431 432

    ++par->childReady;

D
dapan 已提交
433
    SCH_ERR_RET(qSetSubplanExecutionNode(par->plan, task->plan->id.templateId, &task->execAddr));
D
dapan 已提交
434 435
    
    if (SCH_TASK_READY_TO_LUNCH(par)) {
D
dapan 已提交
436
      SCH_ERR_RET(schLaunchTask(job, par));
D
dapan 已提交
437 438 439 440 441 442
    }
  }

  return TSDB_CODE_SUCCESS;
}

D
dapan 已提交
443
int32_t schProcessOnTaskFailure(SSchJob *job, SSchTask *task, int32_t errCode) {
D
dapan 已提交
444
  bool needRetry = false;
D
dapan1121 已提交
445 446
  bool moved = false;
  int32_t taskDone = 0;
D
dapan 已提交
447 448 449 450
  SCH_ERR_RET(schTaskCheckAndSetRetry(job, task, errCode, &needRetry));
  
  if (!needRetry) {
    SCH_TASK_ERR_LOG("task failed[%x], no more retry", errCode);
D
dapan1121 已提交
451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467

    SCH_ERR_RET(schMoveTaskToFailList(job, task, &moved));
    if (!moved) {
      SCH_TASK_ERR_LOG("task may already moved, status:%d", task->status);
    }    
    
    if (SCH_TASK_NEED_WAIT_ALL(task)) {
      SCH_LOCK(SCH_WRITE, &task->level->lock);
      task->level->taskFailed++;
      taskDone = task->level->taskSucceed + task->level->taskFailed;
      SCH_UNLOCK(SCH_WRITE, &task->level->lock);
      
      if (taskDone < task->level->taskNum) {
        qDebug("wait all tasks, done:%d, all:%d", taskDone, task->level->taskNum);
        return TSDB_CODE_SUCCESS;
      }
    }
D
dapan 已提交
468
    
D
dapan1121 已提交
469
    job->status = JOB_TASK_STATUS_FAILED;
D
dapan 已提交
470
    SCH_ERR_RET(schProcessOnJobFailure(job, errCode));
D
dapan 已提交
471 472 473 474

    return TSDB_CODE_SUCCESS;
  }

D
dapan1121 已提交
475
  SCH_ERR_RET(schLaunchTask(job, task));
D
dapan 已提交
476 477 478 479

  return TSDB_CODE_SUCCESS;
}

D
dapan1121 已提交
480
int32_t schProcessRspMsg(SSchJob *job, SSchTask *task, int32_t msgType, char *msg, int32_t msgSize, int32_t rspCode) {
D
dapan1121 已提交
481
  int32_t code = 0;
H
Haojun Liao 已提交
482 483


D
dapan1121 已提交
484
  switch (msgType) {
H
Haojun Liao 已提交
485 486 487 488 489 490 491 492 493 494 495
    case TDMT_VND_CREATE_TABLE_RSP: {
      if (rspCode != TSDB_CODE_SUCCESS) {
        SCH_ERR_JRET(schProcessOnTaskFailure(job, task, rspCode));
      } else {
//        job->resNumOfRows += rsp->affectedRows;
        code = schProcessOnTaskSuccess(job, task);
        if (code) {
          goto _task_error;
        }
      }
    }
D
dapan1121 已提交
496
    case TDMT_VND_SUBMIT_RSP: {
H
Haojun Liao 已提交
497 498
        if (rspCode != TSDB_CODE_SUCCESS) {
          SCH_ERR_JRET(schProcessOnTaskFailure(job, task, rspCode));
D
dapan1121 已提交
499
        } else {
H
Haojun Liao 已提交
500
          SShellSubmitRspMsg *rsp = (SShellSubmitRspMsg *)msg;
D
dapan 已提交
501
          job->resNumOfRows += rsp->affectedRows;
H
Haojun Liao 已提交
502

D
dapan1121 已提交
503 504 505 506
          code = schProcessOnTaskSuccess(job, task);
          if (code) {
            goto _task_error;
          }               
D
dapan1121 已提交
507
        }
D
dapan1121 已提交
508
        break;
D
dapan1121 已提交
509
      }
D
dapan1121 已提交
510
    case TDMT_VND_QUERY_RSP: {
D
dapan1121 已提交
511 512 513 514 515
        SQueryTableRsp *rsp = (SQueryTableRsp *)msg;
        
        if (rsp->code != TSDB_CODE_SUCCESS) {
          SCH_ERR_JRET(schProcessOnTaskFailure(job, task, rsp->code));
        } else {
D
dapan1121 已提交
516
          code = schBuildAndSendMsg(job, task, TDMT_VND_RES_READY);
D
dapan1121 已提交
517 518 519 520 521 522
          if (code) {
            goto _task_error;
          }
        }
        break;
      }
D
dapan1121 已提交
523
    case TDMT_VND_RES_READY_RSP: {
D
dapan1121 已提交
524 525 526 527 528 529 530 531 532 533 534 535
        SResReadyRsp *rsp = (SResReadyRsp *)msg;
        
        if (rsp->code != TSDB_CODE_SUCCESS) {
          SCH_ERR_JRET(schProcessOnTaskFailure(job, task, rsp->code));
        } else {
          code = schProcessOnTaskSuccess(job, task);
          if (code) {
            goto _task_error;
          }        
        }
        break;
      }
D
dapan1121 已提交
536
    case TDMT_VND_FETCH_RSP: {
D
dapan1121 已提交
537 538 539 540 541 542 543 544
        SCH_ERR_JRET(rspCode);
        SRetrieveTableRsp *rsp = (SRetrieveTableRsp *)msg;

        job->res = rsp;
        job->resNumOfRows = rsp->numOfRows;
        
        SCH_ERR_JRET(schProcessOnDataFetched(job));
        break;
D
dapan1121 已提交
545
      }
D
dapan1121 已提交
546 547 548
    case TDMT_VND_DROP_TASK: {

      }
D
dapan1121 已提交
549 550 551 552 553 554 555 556 557 558 559 560
    default:
      qError("unknown msg type:%d received", msgType);
      return TSDB_CODE_QRY_INVALID_INPUT;
  }

  return TSDB_CODE_SUCCESS;

_task_error:
  SCH_ERR_JRET(schProcessOnTaskFailure(job, task, code));
  return TSDB_CODE_SUCCESS;

_return:
D
dapan 已提交
561
  code = schProcessOnJobFailure(job, code);
D
dapan1121 已提交
562 563 564
  return code;
}

D
dapan 已提交
565

D
dapan1121 已提交
566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591
int32_t schHandleCallback(void* param, const SDataBuf* pMsg, int32_t msgType, int32_t rspCode) {
  int32_t code = 0;
  SSchCallbackParam *pParam = (SSchCallbackParam *)param;
  
  SSchJob **job = taosHashGet(schMgmt.jobs, &pParam->queryId, sizeof(pParam->queryId));
  if (NULL == job || NULL == (*job)) {
    qError("taosHashGet queryId:%"PRIx64" not exist", pParam->queryId);
    SCH_ERR_JRET(TSDB_CODE_SCH_INTERNAL_ERROR);
  }

  SSchTask **task = taosHashGet((*job)->execTasks, &pParam->taskId, sizeof(pParam->taskId));
  if (NULL == task || NULL == (*task)) {
    qError("taosHashGet taskId:%"PRIx64" not exist", pParam->taskId);
    SCH_ERR_JRET(TSDB_CODE_SCH_INTERNAL_ERROR);
  }
  
  schProcessRspMsg(*job, *task, msgType, pMsg->pData, pMsg->len, rspCode);

_return:  
  tfree(param);
  SCH_RET(code);
}

int32_t schHandleSubmitCallback(void* param, const SDataBuf* pMsg, int32_t code) {
  return schHandleCallback(param, pMsg, TDMT_VND_SUBMIT_RSP, code);
}
H
Haojun Liao 已提交
592 593 594 595 596

int32_t schHandleCreateTableCallback(void* param, const SDataBuf* pMsg, int32_t code) {
  return schHandleCallback(param, pMsg, TDMT_VND_CREATE_TABLE_RSP, code);
}

D
dapan1121 已提交
597 598 599
int32_t schHandleQueryCallback(void* param, const SDataBuf* pMsg, int32_t code) {
  return schHandleCallback(param, pMsg, TDMT_VND_QUERY_RSP, code);
}
H
Haojun Liao 已提交
600

D
dapan1121 已提交
601 602 603
int32_t schHandleFetchCallback(void* param, const SDataBuf* pMsg, int32_t code) {
  return schHandleCallback(param, pMsg, TDMT_VND_FETCH_RSP, code);
}
H
Haojun Liao 已提交
604

D
dapan1121 已提交
605 606 607
int32_t schHandleReadyCallback(void* param, const SDataBuf* pMsg, int32_t code) {
  return schHandleCallback(param, pMsg, TDMT_VND_RES_READY_RSP, code);
}
H
Haojun Liao 已提交
608

D
dapan1121 已提交
609 610 611 612 613 614 615
int32_t schHandleDropCallback(void* param, const SDataBuf* pMsg, int32_t code) {  
  SSchCallbackParam *pParam = (SSchCallbackParam *)param;
  qDebug("drop task rsp received, queryId:%"PRIx64 ",taksId:%"PRIx64 ",code:%d", pParam->queryId, pParam->taskId, code);
}

int32_t schGetCallbackFp(int32_t msgType, __async_send_cb_fn_t *fp) {
  switch (msgType) {
H
Haojun Liao 已提交
616 617 618
    case TDMT_VND_CREATE_TABLE:
      *fp = schHandleCreateTableCallback;
      break;
D
dapan1121 已提交
619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681
    case TDMT_VND_SUBMIT: 
      *fp = schHandleSubmitCallback;
      break;
    case TDMT_VND_QUERY: 
      *fp = schHandleQueryCallback;
      break;
    case TDMT_VND_RES_READY: 
      *fp = schHandleReadyCallback;
      break;
    case TDMT_VND_FETCH: 
      *fp = schHandleFetchCallback;
      break;
    case TDMT_VND_DROP_TASK:
      *fp = schHandleDropCallback;
      break;
    default:
      qError("unknown msg type:%d", msgType);
      SCH_ERR_RET(TSDB_CODE_QRY_APP_ERROR);
  }

  return TSDB_CODE_SUCCESS;
}


int32_t schAsyncSendMsg(void *transport, SEpSet* epSet, uint64_t qId, uint64_t tId, int32_t msgType, void *msg, uint32_t msgSize) {
  int32_t code = 0;
  SMsgSendInfo* pMsgSendInfo = calloc(1, sizeof(SMsgSendInfo));
  if (NULL == pMsgSendInfo) {
    qError("calloc %d failed", (int32_t)sizeof(SMsgSendInfo));
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  SSchCallbackParam *param = calloc(1, sizeof(SSchCallbackParam));
  if (NULL == param) {
    qError("calloc %d failed", (int32_t)sizeof(SSchCallbackParam));
    SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  __async_send_cb_fn_t fp = NULL;
  SCH_ERR_JRET(schGetCallbackFp(msgType, &fp));

  param->queryId = qId;
  param->taskId = tId;

  pMsgSendInfo->param = param;
  pMsgSendInfo->msgInfo.pData = msg;
  pMsgSendInfo->msgInfo.len = msgSize;
  pMsgSendInfo->msgType = msgType;

  pMsgSendInfo->fp = fp;
  
  int64_t  transporterId = 0;
  SCH_ERR_JRET(asyncSendMsgToServer(transport, epSet, &transporterId, pMsgSendInfo));
  
  return TSDB_CODE_SUCCESS;

_return:
  tfree(param);
  tfree(pMsgSendInfo);

  SCH_RET(code);
}

D
dapan1121 已提交
682 683 684 685 686 687 688 689 690 691
void schConvertAddrToEpSet(SQueryNodeAddr *addr, SEpSet *epSet) {
  epSet->inUse = addr->inUse;
  epSet->numOfEps = addr->numOfEps;
  
  for (int8_t i = 0; i < epSet->numOfEps; ++i) {
    strncpy(epSet->fqdn[i], addr->epAddr[i].fqdn, sizeof(addr->epAddr[i].fqdn));
    epSet->port[i] = addr->epAddr[i].port;
  }
}

D
dapan1121 已提交
692 693 694 695 696 697 698

int32_t schBuildAndSendMsg(SSchJob *job, SSchTask *task, int32_t msgType) {
  uint32_t msgSize = 0;
  void *msg = NULL;
  int32_t code = 0;
  
  switch (msgType) {
H
Haojun Liao 已提交
699
    case TDMT_VND_CREATE_TABLE:
D
dapan1121 已提交
700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723
    case TDMT_VND_SUBMIT: {
      if (NULL == task->msg || task->msgLen <= 0) {
        qError("submit msg is NULL");
        SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
      }

      msgSize = task->msgLen;
      msg = task->msg;
      break;
    }
    case TDMT_VND_QUERY: {
      if (NULL == task->msg) {
        qError("query msg is NULL");
        SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
      }

      msgSize = sizeof(SSubQueryMsg) + task->msgLen;
      msg = calloc(1, msgSize);
      if (NULL == msg) {
        qError("calloc %d failed", msgSize);
        SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
      }

      SSubQueryMsg *pMsg = msg;
D
dapan1121 已提交
724

D
dapan1121 已提交
725
      pMsg->header.vgId = htonl(task->plan->execNode.nodeId);
D
dapan1121 已提交
726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741
      pMsg->sId = htobe64(schMgmt.sId);
      pMsg->queryId = htobe64(job->queryId);
      pMsg->taskId = htobe64(task->taskId);
      pMsg->contentLen = htonl(task->msgLen);
      memcpy(pMsg->msg, task->msg, task->msgLen);
      break;
    }    
    case TDMT_VND_RES_READY: {
      msgSize = sizeof(SResReadyMsg);
      msg = calloc(1, msgSize);
      if (NULL == msg) {
        qError("calloc %d failed", msgSize);
        SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
      }

      SResReadyMsg *pMsg = msg;
D
dapan1121 已提交
742 743
      
      pMsg->header.vgId = htonl(task->plan->execNode.nodeId);      
D
dapan1121 已提交
744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760
      pMsg->sId = htobe64(schMgmt.sId);      
      pMsg->queryId = htobe64(job->queryId);
      pMsg->taskId = htobe64(task->taskId);      
      break;
    }
    case TDMT_VND_FETCH: {
      if (NULL == task) {
        SCH_ERR_RET(TSDB_CODE_QRY_APP_ERROR);
      }
      msgSize = sizeof(SResFetchMsg);
      msg = calloc(1, msgSize);
      if (NULL == msg) {
        qError("calloc %d failed", msgSize);
        SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
      }
    
      SResFetchMsg *pMsg = msg;
D
dapan1121 已提交
761 762
      
      pMsg->header.vgId = htonl(task->plan->execNode.nodeId);            
D
dapan1121 已提交
763 764 765 766 767 768 769 770 771 772 773 774 775 776
      pMsg->sId = htobe64(schMgmt.sId);      
      pMsg->queryId = htobe64(job->queryId);
      pMsg->taskId = htobe64(task->taskId);      
      break;
    }
    case TDMT_VND_DROP_TASK:{
      msgSize = sizeof(STaskDropMsg);
      msg = calloc(1, msgSize);
      if (NULL == msg) {
        qError("calloc %d failed", msgSize);
        SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
      }
    
      STaskDropMsg *pMsg = msg;
D
dapan1121 已提交
777 778
      
      pMsg->header.vgId = htonl(task->plan->execNode.nodeId);            
D
dapan1121 已提交
779 780 781 782 783 784 785 786 787 788 789
      pMsg->sId = htobe64(schMgmt.sId);      
      pMsg->queryId = htobe64(job->queryId);
      pMsg->taskId = htobe64(task->taskId);      
      break;
    }
    default:
      qError("unknown msg type:%d", msgType);
      SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
      break;
  }

D
dapan1121 已提交
790 791 792 793 794 795
  SEpSet epSet;
  SQueryNodeAddr *addr = taosArrayGet(task->condidateAddrs, task->condidateIdx);
  
  schConvertAddrToEpSet(addr, &epSet);

  SCH_ERR_JRET(schAsyncSendMsg(job->transport, &epSet, job->queryId, task->taskId, msgType, msg, msgSize));
D
dapan1121 已提交
796 797 798 799 800 801 802 803

  return TSDB_CODE_SUCCESS;

_return:

  tfree(msg);
  SCH_RET(code);
}
D
dapan1121 已提交
804 805


D
dapan 已提交
806
int32_t schLaunchTask(SSchJob *job, SSchTask *task) {
D
dapan1121 已提交
807
  SSubplan *plan = task->plan;
D
dapan1121 已提交
808
  SCH_ERR_RET(qSubPlanToString(plan, &task->msg, &task->msgLen));
D
dapan1121 已提交
809
  SCH_ERR_RET(schSetTaskCondidateAddrs(job, task));
D
dapan1121 已提交
810

D
dapan1121 已提交
811 812
  if (NULL == task->condidateAddrs || taosArrayGetSize(task->condidateAddrs) <= 0) {
    SCH_TASK_ERR_LOG("no valid condidate node for task:%"PRIx64, task->taskId);
D
dapan1121 已提交
813
    SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR);
D
dapan1121 已提交
814
  }
D
dapan1121 已提交
815

H
Haojun Liao 已提交
816
//  int32_t msgType = (plan->type == QUERY_TYPE_MODIFY)? TDMT_VND_SUBMIT : TDMT_VND_QUERY;
D
dapan 已提交
817
  
H
Haojun Liao 已提交
818
  SCH_ERR_RET(schBuildAndSendMsg(job, task, plan->msgType));
D
dapan 已提交
819
  SCH_ERR_RET(schPushTaskToExecList(job, task));
D
dapan1121 已提交
820

D
dapan1121 已提交
821
  task->status = JOB_TASK_STATUS_EXECUTING;
D
dapan1121 已提交
822 823 824
  return TSDB_CODE_SUCCESS;
}

D
dapan 已提交
825 826
int32_t schLaunchJob(SSchJob *job) {
  SSchLevel *level = taosArrayGet(job->levels, job->levelIdx);
D
dapan 已提交
827
  for (int32_t i = 0; i < level->taskNum; ++i) {
D
dapan 已提交
828
    SSchTask *task = taosArrayGet(level->subTasks, i);
D
dapan1121 已提交
829
    SCH_ERR_RET(schLaunchTask(job, task));
D
dapan1121 已提交
830
  }
D
dapan1121 已提交
831

D
dapan1121 已提交
832
  job->status = JOB_TASK_STATUS_EXECUTING;
D
dapan 已提交
833
  
D
dapan1121 已提交
834
  return TSDB_CODE_SUCCESS;
D
dapan1121 已提交
835 836
}

D
dapan 已提交
837
void schDropJobAllTasks(SSchJob *job) {
D
dapan1121 已提交
838 839
  void *pIter = taosHashIterate(job->succTasks, NULL);
  while (pIter) {
D
dapan 已提交
840
    SSchTask *task = *(SSchTask **)pIter;
D
dapan1121 已提交
841
  
D
dapan1121 已提交
842
    schBuildAndSendMsg(job, task, TDMT_VND_DROP_TASK);
D
dapan1121 已提交
843
    
D
dapan1121 已提交
844 845 846 847 848
    pIter = taosHashIterate(job->succTasks, pIter);
  }  

  pIter = taosHashIterate(job->failTasks, NULL);
  while (pIter) {
D
dapan 已提交
849
    SSchTask *task = *(SSchTask **)pIter;
D
dapan1121 已提交
850
  
D
dapan1121 已提交
851
    schBuildAndSendMsg(job, task, TDMT_VND_DROP_TASK);
D
dapan1121 已提交
852 853
    
    pIter = taosHashIterate(job->succTasks, pIter);
D
dapan1121 已提交
854 855
  }  
}
856

D
dapan1121 已提交
857 858 859 860 861 862 863 864 865 866 867
uint64_t schGenSchId(void) {
  uint64_t sId = 0;

  // TODO

  qDebug("Gen sId:0x%"PRIx64, sId);

  return sId;
}


868
int32_t schedulerInit(SSchedulerCfg *cfg) {
D
dapan1121 已提交
869 870 871 872 873
  if (schMgmt.jobs) {
    qError("scheduler already init");
    return TSDB_CODE_QRY_INVALID_INPUT;
  }

D
dapan 已提交
874 875
  if (cfg) {
    schMgmt.cfg = *cfg;
D
dapan1121 已提交
876 877 878 879
    
    if (schMgmt.cfg.maxJobNum <= 0) {
      schMgmt.cfg.maxJobNum = SCHEDULE_DEFAULT_JOB_NUMBER;
    }
D
dapan1121 已提交
880 881
  } else {
    schMgmt.cfg.maxJobNum = SCHEDULE_DEFAULT_JOB_NUMBER;
D
dapan 已提交
882 883
  }

D
dapan1121 已提交
884 885
  schMgmt.jobs = taosHashInit(schMgmt.cfg.maxJobNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT), false, HASH_ENTRY_LOCK);
  if (NULL == schMgmt.jobs) {
D
dapan1121 已提交
886 887 888
    SCH_ERR_LRET(TSDB_CODE_QRY_OUT_OF_MEMORY, "init %d schduler jobs failed", schMgmt.cfg.maxJobNum);
  }

D
dapan1121 已提交
889
  schMgmt.sId = schGenSchId();
D
dapan1121 已提交
890
  
891 892 893 894
  return TSDB_CODE_SUCCESS;
}


D
dapan1121 已提交
895 896
int32_t scheduleExecJobImpl(void *transport, SArray *nodeList, SQueryDag* pDag, void** pJob, bool syncSchedule) {
  if (nodeList && taosArrayGetSize(nodeList) <= 0) {
D
dapan1121 已提交
897 898 899
    qInfo("qnodeList is empty");
  }

D
dapan1121 已提交
900
  int32_t code = 0;
D
dapan 已提交
901
  SSchJob *job = calloc(1, sizeof(SSchJob));
902
  if (NULL == job) {
D
dapan1121 已提交
903
    SCH_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY);
904 905
  }

D
dapan 已提交
906
  job->attr.syncSchedule = syncSchedule;
D
dapan1121 已提交
907
  job->transport = transport;
D
dapan1121 已提交
908
  job->nodeList = nodeList;
D
dapan 已提交
909

D
dapan1121 已提交
910
  SCH_ERR_JRET(schValidateAndBuildJob(pDag, job));
D
dapan1121 已提交
911

D
dapan 已提交
912 913 914 915 916 917 918 919 920 921 922
  job->execTasks = taosHashInit(pDag->numOfSubplans, taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT), false, HASH_ENTRY_LOCK);
  if (NULL == job->execTasks) {
    qError("taosHashInit %d failed", pDag->numOfSubplans);
    SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

  job->succTasks = taosHashInit(pDag->numOfSubplans, taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT), false, HASH_ENTRY_LOCK);
  if (NULL == job->succTasks) {
    qError("taosHashInit %d failed", pDag->numOfSubplans);
    SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }
D
dapan 已提交
923

D
dapan1121 已提交
924 925 926 927 928 929
  job->failTasks = taosHashInit(pDag->numOfSubplans, taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT), false, HASH_ENTRY_LOCK);
  if (NULL == job->failTasks) {
    qError("taosHashInit %d failed", pDag->numOfSubplans);
    SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY);
  }

D
dapan 已提交
930
  tsem_init(&job->rspSem, 0, 0);
D
dapan1121 已提交
931 932 933 934 935 936 937 938 939 940

  code = taosHashPut(schMgmt.jobs, &job->queryId, sizeof(job->queryId), &job, POINTER_BYTES);
  if (0 != code) {
    if (HASH_NODE_EXIST(code)) {
      qError("taosHashPut queryId:%"PRIx64" already exist", job->queryId);
      SCH_ERR_JRET(TSDB_CODE_QRY_INVALID_INPUT);
    } else {
      qError("taosHashPut queryId:%"PRIx64" failed", job->queryId);
      SCH_ERR_JRET(TSDB_CODE_SCH_INTERNAL_ERROR);
    }
D
dapan1121 已提交
941 942
  }

D
dapan1121 已提交
943
  job->status = JOB_TASK_STATUS_NOT_START;
D
dapan1121 已提交
944
  
D
dapan1121 已提交
945
  SCH_ERR_JRET(schLaunchJob(job));
946

D
dapan 已提交
947
  *(SSchJob **)pJob = job;
D
dapan1121 已提交
948

D
dapan 已提交
949
  if (syncSchedule) {
D
dapan1121 已提交
950 951 952
    tsem_wait(&job->rspSem);
  }

D
dapan1121 已提交
953
  return TSDB_CODE_SUCCESS;
954

D
dapan1121 已提交
955
_return:
956

D
dapan 已提交
957
  *(SSchJob **)pJob = NULL;
D
dapan1121 已提交
958 959 960
  scheduleFreeJob(job);
  
  SCH_RET(code);
961
}
D
dapan1121 已提交
962

D
dapan1121 已提交
963 964
int32_t scheduleExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, void** pJob, SQueryResult *pRes) {
  if (NULL == transport || /* NULL == nodeList || */ NULL == pDag || NULL == pDag->pSubplans || NULL == pJob || NULL == pRes) {
D
dapan1121 已提交
965 966 967
    SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT);
  }

D
dapan1121 已提交
968
  SCH_ERR_RET(scheduleExecJobImpl(transport, nodeList, pDag, pJob, true));
D
dapan1121 已提交
969

D
dapan 已提交
970
  SSchJob *job = *(SSchJob **)pJob;
D
dapan1121 已提交
971
  
D
dapan1121 已提交
972 973 974
  pRes->code = job->errCode;
  pRes->numOfRows = job->resNumOfRows;
  
D
dapan1121 已提交
975 976 977
  return TSDB_CODE_SUCCESS;
}

D
dapan1121 已提交
978 979
int32_t scheduleAsyncExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, void** pJob) {
  if (NULL == transport || NULL == nodeList ||NULL == pDag || NULL == pDag->pSubplans || NULL == pJob) {
D
dapan1121 已提交
980 981 982
    SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT);
  }

D
dapan1121 已提交
983
  return scheduleExecJobImpl(transport, nodeList, pDag, pJob, false);
D
dapan1121 已提交
984 985 986
}


D
dapan1121 已提交
987 988
int32_t scheduleFetchRows(void *pJob, void **data) {
  if (NULL == pJob || NULL == data) {
D
dapan1121 已提交
989
    SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT);
D
dapan 已提交
990 991
  }

D
dapan 已提交
992
  SSchJob *job = pJob;
D
dapan 已提交
993
  int32_t code = 0;
D
dapan 已提交
994

D
dapan1121 已提交
995 996 997 998 999
  if (!job->attr.needFetch) {
    qError("no need to fetch data");
    SCH_ERR_RET(TSDB_CODE_QRY_APP_ERROR);
  }

D
dapan 已提交
1000 1001 1002 1003 1004
  if (job->status == JOB_TASK_STATUS_FAILED) {
    job->res = NULL;
    SCH_RET(job->errCode);
  }

D
dapan1121 已提交
1005 1006 1007 1008 1009
  if (job->status == JOB_TASK_STATUS_SUCCEED) {
    job->res = NULL;
    return TSDB_CODE_SUCCESS;
  }

D
dapan 已提交
1010 1011
  if (atomic_val_compare_exchange_32(&job->userFetch, 0, 1) != 0) {
    qError("prior fetching not finished");
D
dapan1121 已提交
1012
    SCH_ERR_RET(TSDB_CODE_QRY_APP_ERROR);
D
dapan 已提交
1013 1014
  }

D
dapan1121 已提交
1015
  if (job->status == JOB_TASK_STATUS_PARTIAL_SUCCEED) {
D
dapan 已提交
1016 1017
    SCH_ERR_JRET(schFetchFromRemote(job));
  }
D
dapan 已提交
1018 1019 1020

  tsem_wait(&job->rspSem);

D
dapan 已提交
1021 1022 1023 1024 1025 1026 1027 1028
  if (job->status == JOB_TASK_STATUS_FAILED) {
    code = job->errCode;
  }
  
  if (job->res && ((SRetrieveTableRsp *)job->res)->completed) {
    job->status = JOB_TASK_STATUS_SUCCEED;
  }

D
dapan 已提交
1029
  *data = job->res;
D
dapan 已提交
1030
  job->res = NULL;
D
dapan 已提交
1031

D
dapan 已提交
1032 1033 1034
_return:
  atomic_val_compare_exchange_32(&job->userFetch, 1, 0);

D
dapan1121 已提交
1035
  SCH_RET(code);
D
dapan 已提交
1036
}
D
dapan1121 已提交
1037

D
dapan1121 已提交
1038 1039
int32_t scheduleCancelJob(void *pJob) {
  //TODO
D
dapan1121 已提交
1040

D
dapan1121 已提交
1041 1042
  //TODO MOVE ALL TASKS FROM EXEC LIST TO FAIL LIST

D
dapan1121 已提交
1043 1044 1045 1046 1047
  return TSDB_CODE_SUCCESS;
}

void scheduleFreeJob(void *pJob) {
  if (NULL == pJob) {
D
dapan 已提交
1048 1049
    return;
  }
D
dapan1121 已提交
1050

D
dapan 已提交
1051
  SSchJob *job = pJob;
D
dapan1121 已提交
1052 1053

  if (job->status > 0) {
D
dapan1121 已提交
1054
    if (0 != taosHashRemove(schMgmt.jobs, &job->queryId, sizeof(job->queryId))) {
D
dapan1121 已提交
1055 1056 1057 1058
      qError("remove job:%"PRIx64"from mgmt failed", job->queryId); // maybe already freed
      return;
    }

D
dapan1121 已提交
1059
    if (job->status == JOB_TASK_STATUS_EXECUTING) {
D
dapan1121 已提交
1060 1061
      scheduleCancelJob(pJob);
    }
D
dapan1121 已提交
1062 1063

    schDropJobAllTasks(job);
D
dapan1121 已提交
1064 1065 1066
  }
  
  //TODO free job
D
dapan1121 已提交
1067 1068
}

1069
void schedulerDestroy(void) {
D
dapan1121 已提交
1070 1071 1072
  if (schMgmt.jobs) {
    taosHashCleanup(schMgmt.jobs); //TODO
    schMgmt.jobs = NULL;
1073 1074 1075
  }
}

D
dapan1121 已提交
1076