public QueryStatus getQueryStatus(QueryId queryId) throws ServiceException {
    GetQueryStatusRequest.Builder builder = GetQueryStatusRequest.newBuilder();
    builder.setQueryId(queryId.getProto());

    GetQueryStatusResponse res = null;
    if (queryMasterMap.containsKey(queryId)) {
      NettyClientBase qmClient = null;
      try {
        qmClient =
            connPool.getConnection(
                queryMasterMap.get(queryId), QueryMasterClientProtocol.class, false);
        QueryMasterClientProtocolService.BlockingInterface queryMasterService = qmClient.getStub();
        res = queryMasterService.getQueryStatus(null, builder.build());
      } catch (Exception e) {
        throw new ServiceException(e.getMessage(), e);
      } finally {
        connPool.releaseConnection(qmClient);
      }
    } else {
      NettyClientBase tmClient = null;
      try {
        tmClient = connPool.getConnection(tajoMasterAddr, TajoMasterClientProtocol.class, false);
        TajoMasterClientProtocolService.BlockingInterface tajoMasterService = tmClient.getStub();
        res = tajoMasterService.getQueryStatus(null, builder.build());

        String queryMasterHost = res.getQueryMasterHost();
        if (queryMasterHost != null && !queryMasterHost.isEmpty()) {
          NettyClientBase qmClient = null;
          try {
            InetSocketAddress qmAddr =
                NetUtils.createSocketAddr(queryMasterHost, res.getQueryMasterPort());
            qmClient = connPool.getConnection(qmAddr, QueryMasterClientProtocol.class, false);
            QueryMasterClientProtocolService.BlockingInterface queryMasterService =
                qmClient.getStub();
            res = queryMasterService.getQueryStatus(null, builder.build());

            queryMasterMap.put(queryId, qmAddr);
          } catch (Exception e) {
            throw new ServiceException(e.getMessage(), e);
          } finally {
            connPool.releaseConnection(qmClient);
          }
        }
      } catch (Exception e) {
        throw new ServiceException(e.getMessage(), e);
      } finally {
        connPool.releaseConnection(tmClient);
      }
    }
    return new QueryStatus(res);
  }
  public void reportQueryStatusToQueryMaster(QueryId queryId, TajoProtos.QueryState state) {
    LOG.info("Send QueryMaster Ready to QueryJobManager:" + queryId);
    NettyClientBase tmClient = null;
    try {
      tmClient =
          connPool.getConnection(
              queryMasterContext.getWorkerContext().getTajoMasterAddress(),
              TajoMasterProtocol.class,
              true);
      TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();

      TajoHeartbeat.Builder queryHeartbeatBuilder =
          TajoHeartbeat.newBuilder()
              .setTajoWorkerHost(
                  workerContext.getQueryMasterManagerService().getBindAddr().getHostName())
              .setTajoQueryMasterPort(
                  workerContext.getQueryMasterManagerService().getBindAddr().getPort())
              .setTajoWorkerClientPort(
                  workerContext.getTajoWorkerClientService().getBindAddr().getPort())
              .setState(state)
              .setQueryId(queryId.getProto());

      CallFuture<TajoHeartbeatResponse> callBack = new CallFuture<TajoHeartbeatResponse>();

      masterClientService.heartbeat(
          callBack.getController(), queryHeartbeatBuilder.build(), callBack);
    } catch (Exception e) {
      LOG.error(e.getMessage(), e);
    } finally {
      connPool.releaseConnection(tmClient);
    }
  }
  public List<TajoMasterProtocol.WorkerResourceProto> getAllWorker() {

    NettyClientBase rpc = null;
    try {
      rpc =
          connPool.getConnection(
              queryMasterContext.getWorkerContext().getTajoMasterAddress(),
              TajoMasterProtocol.class,
              true);
      TajoMasterProtocol.TajoMasterProtocolService masterService = rpc.getStub();

      CallFuture<TajoMasterProtocol.WorkerResourcesRequest> callBack =
          new CallFuture<TajoMasterProtocol.WorkerResourcesRequest>();
      masterService.getAllWorkerResource(
          callBack.getController(), PrimitiveProtos.NullProto.getDefaultInstance(), callBack);

      TajoMasterProtocol.WorkerResourcesRequest workerResourcesRequest =
          callBack.get(2, TimeUnit.SECONDS);
      return workerResourcesRequest.getWorkerResourcesList();
    } catch (Exception e) {
      LOG.error(e.getMessage(), e);
    } finally {
      connPool.releaseConnection(rpc);
    }
    return new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
  }
  public boolean killQuery(final QueryId queryId) throws ServiceException, IOException {

    QueryStatus status = getQueryStatus(queryId);

    NettyClientBase tmClient = null;
    try {
      /* send a kill to the TM */
      tmClient = connPool.getConnection(tajoMasterAddr, TajoMasterClientProtocol.class, false);
      TajoMasterClientProtocolService.BlockingInterface tajoMasterService = tmClient.getStub();
      tajoMasterService.killQuery(null, queryId.getProto());

      long currentTimeMillis = System.currentTimeMillis();
      long timeKillIssued = currentTimeMillis;
      while ((currentTimeMillis < timeKillIssued + 10000L)
          && (status.getState() != QueryState.QUERY_KILLED)) {
        try {
          Thread.sleep(100L);
        } catch (InterruptedException ie) {
          break;
        }
        currentTimeMillis = System.currentTimeMillis();
        status = getQueryStatus(queryId);
      }
      return status.getState() == QueryState.QUERY_KILLED;
    } catch (Exception e) {
      LOG.debug("Error when checking for application status", e);
      return false;
    } finally {
      connPool.releaseConnection(tmClient);
    }
  }
  public GetQueryResultResponse getResultResponse(QueryId queryId) throws ServiceException {
    if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
      return null;
    }

    NettyClientBase client = null;
    try {
      InetSocketAddress queryMasterAddr = queryMasterMap.get(queryId);
      if (queryMasterAddr == null) {
        LOG.warn("No Connection to QueryMaster for " + queryId);
        return null;
      }
      client = connPool.getConnection(queryMasterAddr, QueryMasterClientProtocol.class, false);
      QueryMasterClientProtocolService.BlockingInterface queryMasterService = client.getStub();
      GetQueryResultRequest.Builder builder = GetQueryResultRequest.newBuilder();
      builder.setQueryId(queryId.getProto());
      GetQueryResultResponse response = queryMasterService.getQueryResult(null, builder.build());

      return response;
    } catch (Exception e) {
      throw new ServiceException(e.getMessage(), e);
    } finally {
      connPool.releaseConnection(client);
    }
  }
  private void cleanup(QueryId queryId) {
    LOG.info("cleanup query resources : " + queryId);
    NettyClientBase rpc = null;
    List<TajoMasterProtocol.WorkerResourceProto> workers = getAllWorker();

    for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
      try {
        if (worker.getPeerRpcPort() == 0) continue;

        rpc =
            connPool.getConnection(
                NetUtils.createSocketAddr(worker.getHost(), worker.getPeerRpcPort()),
                TajoWorkerProtocol.class,
                true);
        TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub();

        tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get());
      } catch (Exception e) {
        LOG.error(e.getMessage());
      } finally {
        connPool.releaseConnection(rpc);
      }
    }
  }
    @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");
    }