protected int getNumWorkers(WorkerStatus status) {
    int numWorkers = 0;
    for (WorkerResource eachWorker : tajoMasterContext.getResourceManager().getWorkers().values()) {
      if (eachWorker.getWorkerStatus() == status) {
        numWorkers++;
      }
    }

    return numWorkers;
  }
  private Tuple getWorkerTuple(Schema outSchema, Worker aWorker) {
    List<Column> columns = outSchema.getRootColumns();
    Tuple aTuple = new VTuple(outSchema.size());
    WorkerResource aResource = aWorker.getResource();

    for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
      Column column = columns.get(fieldId);

      if ("host".equalsIgnoreCase(column.getSimpleName())) {
        if (aWorker.getConnectionInfo() != null && aWorker.getConnectionInfo().getHost() != null) {
          aTuple.put(fieldId, DatumFactory.createText(aWorker.getConnectionInfo().getHost()));
        } else {
          aTuple.put(fieldId, DatumFactory.createNullDatum());
        }
      } else if ("port".equalsIgnoreCase(column.getSimpleName())) {
        if (aWorker.getConnectionInfo() != null) {
          aTuple.put(
              fieldId, DatumFactory.createInt4(aWorker.getConnectionInfo().getPeerRpcPort()));
        } else {
          aTuple.put(fieldId, DatumFactory.createNullDatum());
        }
      } else if ("type".equalsIgnoreCase(column.getSimpleName())) {
        aTuple.put(fieldId, DatumFactory.createText("Worker"));
      } else if ("status".equalsIgnoreCase(column.getSimpleName())) {
        aTuple.put(fieldId, DatumFactory.createText(aWorker.getState().toString()));
      } else if ("RUNNING".equalsIgnoreCase(aWorker.getState().toString())) {
        if ("total_cpu".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createInt4(aResource.getCpuCoreSlots()));
        } else if ("used_mem".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createInt8(aResource.getUsedMemoryMB() * 1048576l));
        } else if ("total_mem".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMemoryMB() * 1048576l));
        } else if ("free_heap".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createInt8(aResource.getFreeHeap()));
        } else if ("max_heap".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createInt8(aResource.getMaxHeap()));
        } else if ("used_diskslots".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createFloat4(aResource.getUsedDiskSlots()));
        } else if ("total_diskslots".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createFloat4(aResource.getDiskSlots()));
        } else if ("running_tasks".equalsIgnoreCase(column.getSimpleName())) {
          aTuple.put(fieldId, DatumFactory.createInt4(aResource.getNumRunningTasks()));
        } else if ("last_heartbeat_ts".equalsIgnoreCase(column.getSimpleName())) {
          if (aWorker.getLastHeartbeatTime() > 0) {
            aTuple.put(
                fieldId,
                DatumFactory.createTimestmpDatumWithJavaMillis(aWorker.getLastHeartbeatTime()));
          } else {
            aTuple.put(fieldId, DatumFactory.createNullDatum());
          }
        }
      } else {
        aTuple.put(fieldId, DatumFactory.createNullDatum());
      }
    }

    return aTuple;
  }
    @Override
    public void run() {
      LOG.info("Start TajoWorkerAllocationThread");
      CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack =
          new CallFuture<TajoMasterProtocol.WorkerResourceAllocationResponse>();

      int requiredMemoryMBSlot = 512; // TODO
      int requiredDiskSlots = 1; // TODO
      TajoMasterProtocol.WorkerResourceAllocationRequest request =
          TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
              .setMemoryMBSlots(requiredMemoryMBSlot)
              .setDiskSlots(requiredDiskSlots)
              .setNumWorks(event.getRequiredNum())
              .setExecutionBlockId(event.getExecutionBlockId().getProto())
              .build();

      RpcConnectionPool connPool = RpcConnectionPool.getPool(queryTaskContext.getConf());
      NettyClientBase tmClient = null;
      try {
        tmClient =
            connPool.getConnection(
                queryTaskContext.getQueryMasterContext().getWorkerContext().getTajoMasterAddress(),
                TajoMasterProtocol.class,
                true);
        TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
        masterClientService.allocateWorkerResources(null, request, callBack);
      } catch (Exception e) {
        connPool.closeConnection(tmClient);
        tmClient = null;
        LOG.error(e.getMessage(), e);
      } finally {
        connPool.releaseConnection(tmClient);
      }

      TajoMasterProtocol.WorkerResourceAllocationResponse response = null;
      while (!stopped.get()) {
        try {
          response = callBack.get(3, TimeUnit.SECONDS);
          break;
        } catch (InterruptedException e) {
          if (stopped.get()) {
            return;
          }
        } catch (TimeoutException e) {
          LOG.info("No available worker resource for " + event.getExecutionBlockId());
          continue;
        }
      }
      int numAllocatedWorkers = 0;

      if (response != null) {
        List<TajoMasterProtocol.WorkerAllocatedResource> workerHosts =
            response.getWorkerAllocatedResourceList();
        ExecutionBlockId executionBlockId = event.getExecutionBlockId();

        List<Container> containers = new ArrayList<Container>();
        for (TajoMasterProtocol.WorkerAllocatedResource eachWorker : workerHosts) {
          TajoWorkerContainer container = new TajoWorkerContainer();
          NodeIdPBImpl nodeId = new NodeIdPBImpl();

          nodeId.setHost(eachWorker.getWorkerHost());
          nodeId.setPort(eachWorker.getPeerRpcPort());

          TajoWorkerContainerId containerId = new TajoWorkerContainerId();

          containerId.setApplicationAttemptId(
              ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId()));
          containerId.setId(containerIdSeq.incrementAndGet());

          container.setId(containerId);
          container.setNodeId(nodeId);

          WorkerResource workerResource = new WorkerResource();
          workerResource.setAllocatedHost(nodeId.getHost());
          workerResource.setPeerRpcPort(nodeId.getPort());
          workerResource.setQueryMasterPort(eachWorker.getQueryMasterPort());
          workerResource.setPullServerPort(eachWorker.getWorkerPullServerPort());
          workerResource.setMemoryMBSlots(requiredMemoryMBSlot);
          workerResource.setDiskSlots(requiredDiskSlots);

          container.setWorkerResource(workerResource);

          containers.add(container);
        }

        SubQueryState state = queryTaskContext.getSubQuery(executionBlockId).getState();
        if (!SubQuery.isRunningState(state)) {
          List<WorkerResource> workerResources = new ArrayList<WorkerResource>();
          for (Container eachContainer : containers) {
            workerResources.add(((TajoWorkerContainer) eachContainer).getWorkerResource());
          }
          try {
            TajoContainerProxy.releaseWorkerResource(
                queryTaskContext, executionBlockId, workerResources);
          } catch (Exception e) {
            LOG.error(e.getMessage(), e);
          }
          return;
        }

        if (workerHosts.size() > 0) {
          if (LOG.isDebugEnabled()) {
            LOG.debug("SubQueryContainerAllocationEvent fire:" + executionBlockId);
          }
          queryTaskContext
              .getEventHandler()
              .handle(new SubQueryContainerAllocationEvent(executionBlockId, containers));
        }
        numAllocatedWorkers += workerHosts.size();
      }
      if (event.getRequiredNum() > numAllocatedWorkers) {
        ContainerAllocationEvent shortRequestEvent =
            new ContainerAllocationEvent(
                event.getType(),
                event.getExecutionBlockId(),
                event.getPriority(),
                event.getResource(),
                event.getRequiredNum() - numAllocatedWorkers,
                event.isLeafQuery(),
                event.getProgress());
        queryTaskContext.getEventHandler().handle(shortRequestEvent);
      }
      LOG.info("Stop TajoWorkerAllocationThread");
    }