You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2013/08/26 14:29:10 UTC

[1/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Updated Branches:
  refs/heads/master 8b8b6683d -> d48f2667b


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml b/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
index c003637..20b80aa 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
+++ b/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
@@ -35,4 +35,10 @@
     <name>tajo.task.localdir</name>
     <value>/tmp/tajo-localdir</value>
   </property>
+
+  <property>
+    <name>tajo.resource.manager</name>
+    <value>org.apache.tajo.master.rm.TajoWorkerResourceManager</value>
+    <description>org.apache.tajo.master.rm.TajoWorkerResourceManager or org.apache.tajo.master.rm.YarnTajoResourceManager</description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/pom.xml b/tajo-core/tajo-core-pullserver/pom.xml
index 1ce6aba..b46caba 100644
--- a/tajo-core/tajo-core-pullserver/pom.xml
+++ b/tajo-core/tajo-core-pullserver/pom.xml
@@ -63,6 +63,7 @@
       <artifactId>netty</artifactId>
       <scope>compile</scope>
     </dependency>
+
     <dependency>
       <groupId>commons-logging</groupId>
       <artifactId>commons-logging</artifactId>
@@ -154,16 +155,6 @@
   <reporting>
     <plugins>
       <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-        <version>2.3.2</version>
-        <configuration>
-          <xmlOutput>true</xmlOutput>
-          <xmlOutputDirectory>target/findbugs</xmlOutputDirectory>
-          <!--<excludeFilterFile>src/main/findbugs/findbugs-exclude.xml</excludeFilterFile>-->
-        </configuration>
-      </plugin>
-      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-project-info-reports-plugin</artifactId>
         <version>2.4</version>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
index 1aae489..31db15c 100644
--- a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
@@ -22,6 +22,9 @@ import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.pullserver.retriever.DataRetriever;
+import org.apache.tajo.pullserver.retriever.FileChunk;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.*;
 import org.jboss.netty.handler.codec.frame.TooLongFrameException;
@@ -29,9 +32,6 @@ import org.jboss.netty.handler.codec.http.*;
 import org.jboss.netty.handler.ssl.SslHandler;
 import org.jboss.netty.handler.stream.ChunkedFile;
 import org.jboss.netty.util.CharsetUtil;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.pullserver.retriever.DataRetriever;
-import org.apache.tajo.pullserver.retriever.FileChunk;
 
 import java.io.*;
 import java.net.URLDecoder;
@@ -51,7 +51,8 @@ import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
 public class HttpDataServerHandler extends SimpleChannelUpstreamHandler {
   private final static Log LOG = LogFactory.getLog(HttpDataServerHandler.class);
 
-  Map<SubQueryId, DataRetriever> retrievers = new ConcurrentHashMap<SubQueryId, DataRetriever>();
+  Map<ExecutionBlockId, DataRetriever> retrievers =
+      new ConcurrentHashMap<ExecutionBlockId, DataRetriever>();
   private String userName;
   private String appId;
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
index 0bdf504..924289e 100644
--- a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices;
 import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
@@ -51,6 +52,7 @@ import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.storage.TupleComparator;
 import org.apache.tajo.storage.TupleRange;
 import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.util.TajoIdUtils;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.*;
@@ -104,6 +106,7 @@ public class PullServerAuxService extends AbstractService
   private int sslFileBufferSize;
 
   private ApplicationId appId;
+  private QueryId queryId;
   private FileSystem localFS;
 
   /**
@@ -189,6 +192,7 @@ public class PullServerAuxService extends AbstractService
     // TODO these bytes should be versioned
     // TODO: Once SHuffle is out of NM, this can use MR APIs
     this.appId = appId;
+    this.queryId = TajoIdUtils.parseQueryId(appId.toString());
     this.userName = user;
     userRsrc.put(appId.toString(), user);
   }
@@ -250,6 +254,10 @@ public class PullServerAuxService extends AbstractService
                                     DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
   }
 
+  public int getPort() {
+    return port;
+  }
+
   @Override
   public synchronized void stop() {
     try {
@@ -384,7 +392,12 @@ public class PullServerAuxService extends AbstractService
       List<String> taskIds = splitMaps(taskIdList);
 
       // the working dir of tajo worker for each query
-      String queryBaseDir = appId + "/output" + "/";
+      String queryBaseDir = queryId + "/output" + "/";
+
+      LOG.info("PullServer request param: repartitionType=" + repartitionType +
+          ", sid=" + sid + ", partitionId=" + partitionId + ", taskIds=" + taskIdList);
+
+      LOG.info("PullServer baseDir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname) + "/" + queryBaseDir);
 
       // if a subquery requires a range partitioning
       if (repartitionType.equals("r")) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
new file mode 100644
index 0000000..3df3672
--- /dev/null
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
@@ -0,0 +1,652 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.pullserver.retriever.FileChunk;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.*;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+public class TajoPullServerService extends AbstractService {
+
+  private static final Log LOG = LogFactory.getLog(TajoPullServerService.class);
+
+  public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache";
+  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+  public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes";
+  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+  private int port;
+  private ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  private HttpPipelineFactory pipelineFact;
+  private int sslFileBufferSize;
+
+  private ApplicationId appId;
+  private FileSystem localFS;
+
+  /**
+   * Should the shuffle use posix_fadvise calls to manage the OS cache during
+   * sendfile
+   */
+  private boolean manageOsCache;
+  private int readaheadLength;
+  private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+
+
+  public static final String PULLSERVER_SERVICEID = "tajo.pullserver";
+
+  private static final Map<String,String> userRsrc =
+    new ConcurrentHashMap<String,String>();
+  private static String userName;
+
+  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+    "tajo.pullserver.ssl.file.buffer.size";
+
+  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+  @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo")
+  static class ShuffleMetrics implements ChannelFutureListener {
+    @Metric({"OutputBytes","PullServer output in bytes"})
+    MutableCounterLong shuffleOutputBytes;
+    @Metric({"Failed","# of failed shuffle outputs"})
+    MutableCounterInt shuffleOutputsFailed;
+    @Metric({"Succeeded","# of succeeded shuffle outputs"})
+    MutableCounterInt shuffleOutputsOK;
+    @Metric({"Connections","# of current shuffle connections"})
+    MutableGaugeInt shuffleConnections;
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isSuccess()) {
+        shuffleOutputsOK.incr();
+      } else {
+        shuffleOutputsFailed.incr();
+      }
+      shuffleConnections.decr();
+    }
+  }
+
+  final ShuffleMetrics metrics;
+
+  TajoPullServerService(MetricsSystem ms) {
+    super("httpshuffle");
+    metrics = ms.register(new ShuffleMetrics());
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public TajoPullServerService() {
+    this(DefaultMetricsSystem.instance());
+  }
+
+  /**
+   * Serialize the shuffle port into a ByteBuffer for use later on.
+   * @param port the port to be sent to the ApplciationMaster
+   * @return the serialized form of the port.
+   */
+  public static ByteBuffer serializeMetaData(int port) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer port_dob = new DataOutputBuffer();
+    port_dob.writeInt(port);
+    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+  }
+
+  /**
+   * A helper function to deserialize the metadata returned by PullServerAuxService.
+   * @param meta the metadata returned by the PullServerAuxService
+   * @return the port the PullServer Handler is listening on to serve shuffle data.
+   */
+  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+    //TODO this should be returning a class not just an int
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    return in.readInt();
+  }
+
+  public void initApp(String user, ApplicationId appId, ByteBuffer secret) {
+    // TODO these bytes should be versioned
+    // TODO: Once SHuffle is out of NM, this can use MR APIs
+    this.appId = appId;
+    this.userName = user;
+    userRsrc.put(appId.toString(), user);
+  }
+
+  public void stopApp(ApplicationId appId) {
+    userRsrc.remove(appId.toString());
+  }
+
+  @Override
+  public synchronized void init(Configuration conf) {
+    try {
+      manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+          DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+      readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+          DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+      ThreadFactory bossFactory = new ThreadFactoryBuilder()
+          .setNameFormat("PullServerAuxService Netty Boss #%d")
+          .build();
+      ThreadFactory workerFactory = new ThreadFactoryBuilder()
+          .setNameFormat("PullServerAuxService Netty Worker #%d")
+          .build();
+
+      selector = new NioServerSocketChannelFactory(
+          Executors.newCachedThreadPool(bossFactory),
+          Executors.newCachedThreadPool(workerFactory));
+
+      localFS = new LocalFileSystem();
+      super.init(new Configuration(conf));
+    } catch (Throwable t) {
+      LOG.error(t);
+    }
+  }
+
+  // TODO change AbstractService to throw InterruptedException
+  @Override
+  public synchronized void start() {
+    Configuration conf = getConfig();
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+    port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+        ConfVars.PULLSERVER_PORT.defaultIntVal);
+    Channel ch = bootstrap.bind(new InetSocketAddress(port));
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port));
+    pipelineFact.PullServer.setPort(port);
+    LOG.info(getName() + " listening on port " + port);
+    super.start();
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  @Override
+  public synchronized void stop() {
+    try {
+      accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+      ServerBootstrap bootstrap = new ServerBootstrap(selector);
+      bootstrap.releaseExternalResources();
+      pipelineFact.destroy();
+
+      localFS.close();
+    } catch (Throwable t) {
+      LOG.error(t);
+    } finally {
+      super.stop();
+    }
+  }
+
+  public synchronized ByteBuffer getMeta() {
+    try {
+      return serializeMetaData(port); 
+    } catch (IOException e) {
+      LOG.error("Error during getMeta", e);
+      // TODO add API to AuxiliaryServices to report failures
+      return null;
+    }
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final PullServer PullServer;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      PullServer = new PullServer(conf);
+      if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname,
+          ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+        sslFactory.init();
+      }
+    }
+
+    public void destroy() {
+      if (sslFactory != null) {
+        sslFactory.destroy();
+      }
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = Channels.pipeline();
+      if (sslFactory != null) {
+        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+      }
+      pipeline.addLast("decoder", new HttpRequestDecoder());
+      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+      pipeline.addLast("encoder", new HttpResponseEncoder());
+      pipeline.addLast("chunking", new ChunkedWriteHandler());
+      pipeline.addLast("shuffle", PullServer);
+      return pipeline;
+      // TODO factor security manager into pipeline
+      // TODO factor out encode/decode to permit binary shuffle
+      // TODO factor out decode of index to permit alt. models
+    }
+  }
+
+  class PullServer extends SimpleChannelUpstreamHandler {
+
+    private final Configuration conf;
+//    private final IndexCache indexCache;
+    private final LocalDirAllocator lDirAlloc =
+      new LocalDirAllocator(ConfVars.TASK_LOCAL_DIR.varname);
+    private int port;
+
+    public PullServer(Configuration conf) {
+      this.conf = conf;
+//      indexCache = new IndexCache(new JobConf(conf));
+      this.port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+          ConfVars.PULLSERVER_PORT.defaultIntVal);
+    }
+    
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    private List<String> splitMaps(List<String> mapq) {
+      if (null == mapq) {
+        return null;
+      }
+      final List<String> ret = new ArrayList<String>();
+      for (String s : mapq) {
+        Collections.addAll(ret, s.split(","));
+      }
+      return ret;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+        throws Exception {
+
+      HttpRequest request = (HttpRequest) e.getMessage();
+      if (request.getMethod() != GET) {
+        sendError(ctx, METHOD_NOT_ALLOWED);
+        return;
+      }
+
+      // Parsing the URL into key-values
+      final Map<String, List<String>> params =
+          new QueryStringDecoder(request.getUri()).getParameters();
+      final List<String> types = params.get("type");
+      final List<String> qids = params.get("qid");
+      final List<String> taskIdList = params.get("ta");
+      final List<String> subQueryIds = params.get("sid");
+      final List<String> partitionIds = params.get("p");
+
+      if (types == null || taskIdList == null || subQueryIds == null || qids == null
+          || partitionIds == null) {
+        sendError(ctx, "Required queryId, type, taskIds, subquery Id, and partition id",
+            BAD_REQUEST);
+        return;
+      }
+
+      if (qids.size() != 1 && types.size() != 1 || subQueryIds.size() != 1) {
+        sendError(ctx, "Required qids, type, taskIds, subquery Id, and partition id",
+            BAD_REQUEST);
+        return;
+      }
+
+      final List<FileChunk> chunks = Lists.newArrayList();
+
+      String queryId = qids.get(0);
+      String repartitionType = types.get(0);
+      String sid = subQueryIds.get(0);
+      String partitionId = partitionIds.get(0);
+      List<String> taskIds = splitMaps(taskIdList);
+
+      LOG.info("PullServer request param: repartitionType=" + repartitionType +
+          ", sid=" + sid + ", partitionId=" + partitionId + ", taskIds=" + taskIdList);
+
+      // the working dir of tajo worker for each query
+      String queryBaseDir = queryId.toString() + "/output";
+
+      LOG.info("PullServer baseDir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname) + "/" + queryBaseDir);
+
+      // if a subquery requires a range partitioning
+      if (repartitionType.equals("r")) {
+        String ta = taskIds.get(0);
+        Path path = localFS.makeQualified(
+            lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf));
+
+        String startKey = params.get("start").get(0);
+        String endKey = params.get("end").get(0);
+        boolean last = params.get("final") != null;
+
+        FileChunk chunk;
+        try {
+          chunk = getFileCunks(path, startKey, endKey, last);
+        } catch (Throwable t) {
+          LOG.error("ERROR Request: " + request.getUri(), t);
+          sendError(ctx, "Cannot get file chunks to be sent", BAD_REQUEST);
+          return;
+        }
+        if (chunk != null) {
+          chunks.add(chunk);
+        }
+
+        // if a subquery requires a hash repartition
+      } else if (repartitionType.equals("h")) {
+        for (String ta : taskIds) {
+          Path path = localFS.makeQualified(
+              lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" +
+                  ta + "/output/" + partitionId, conf));
+          File file = new File(path.toUri());
+          FileChunk chunk = new FileChunk(file, 0, file.length());
+          chunks.add(chunk);
+        }
+      } else {
+        LOG.error("Unknown repartition type: " + repartitionType);
+        return;
+      }
+
+      // Write the content.
+      Channel ch = e.getChannel();
+      if (chunks.size() == 0) {
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
+        ch.write(response);
+        if (!isKeepAlive(request)) {
+          ch.close();
+        }
+      }  else {
+        FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
+        HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+        long totalSize = 0;
+        for (FileChunk chunk : file) {
+          totalSize += chunk.length();
+        }
+        setContentLength(response, totalSize);
+
+        // Write the initial line and the header.
+        ch.write(response);
+
+        ChannelFuture writeFuture = null;
+
+        for (FileChunk chunk : file) {
+          writeFuture = sendFile(ctx, ch, chunk);
+          if (writeFuture == null) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        }
+
+        // Decide whether to close the connection or not.
+        if (!isKeepAlive(request)) {
+          // Close the connection when the whole content is written out.
+          writeFuture.addListener(ChannelFutureListener.CLOSE);
+        }
+      }
+    }
+
+    private ChannelFuture sendFile(ChannelHandlerContext ctx,
+                                   Channel ch,
+                                   FileChunk file) throws IOException {
+      RandomAccessFile spill;
+      try {
+        spill = new RandomAccessFile(file.getFile(), "r");
+      } catch (FileNotFoundException e) {
+        LOG.info(file.getFile() + " not found");
+        return null;
+      }
+      ChannelFuture writeFuture;
+      if (ch.getPipeline().get(SslHandler.class) == null) {
+        final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
+            file.startOffset, file.length(), manageOsCache, readaheadLength,
+            readaheadPool, file.getFile().getAbsolutePath());
+        writeFuture = ch.write(partition);
+        writeFuture.addListener(new ChannelFutureListener() {
+          // TODO error handling; distinguish IO/connection failures,
+          //      attribute to appropriate spill output
+          @Override
+          public void operationComplete(ChannelFuture future) {
+            partition.releaseExternalResources();
+          }
+        });
+      } else {
+        // HTTPS cannot be done with zero copy.
+        final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+            file.startOffset, file.length, sslFileBufferSize,
+            manageOsCache, readaheadLength, readaheadPool,
+            file.getFile().getAbsolutePath());
+        writeFuture = ch.write(chunk);
+      }
+      metrics.shuffleConnections.incr();
+      metrics.shuffleOutputBytes.incr(file.length); // optimistic
+      return writeFuture;
+    }
+
+    private void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    private void sendError(ChannelHandlerContext ctx, String message,
+        HttpResponseStatus status) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+      response.setContent(
+        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+
+      // Close the connection as soon as the error message is sent.
+      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      Channel ch = e.getChannel();
+      Throwable cause = e.getCause();
+      if (cause instanceof TooLongFrameException) {
+        sendError(ctx, BAD_REQUEST);
+        return;
+      }
+
+      LOG.error("PullServer error: ", cause);
+      if (ch.isConnected()) {
+        LOG.error("PullServer error " + e);
+        sendError(ctx, INTERNAL_SERVER_ERROR);
+      }
+    }
+  }
+
+  public FileChunk getFileCunks(Path outDir,
+                                      String startKey,
+                                      String endKey,
+                                      boolean last) throws IOException {
+    BSTIndex index = new BSTIndex(new TajoConf());
+    BSTIndex.BSTIndexReader idxReader =
+        index.getIndexReader(new Path(outDir, "index"));
+    idxReader.open();
+    Schema keySchema = idxReader.getKeySchema();
+    TupleComparator comparator = idxReader.getComparator();
+
+    LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
+        + idxReader.getLastKey());
+
+    File data = new File(URI.create(outDir.toUri() + "/output"));
+    byte [] startBytes = Base64.decodeBase64(startKey);
+    byte [] endBytes = Base64.decodeBase64(endKey);
+
+    Tuple start;
+    Tuple end;
+    try {
+      start = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, startBytes);
+    } catch (Throwable t) {
+      throw new IllegalArgumentException("StartKey: " + startKey
+          + ", decoded byte size: " + startBytes.length, t);
+    }
+
+    try {
+      end = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, endBytes);
+    } catch (Throwable t) {
+      throw new IllegalArgumentException("EndKey: " + endKey
+          + ", decoded byte size: " + endBytes.length, t);
+    }
+
+
+    if(!comparator.isAscendingFirstKey()) {
+      Tuple tmpKey = start;
+      start = end;
+      end = tmpKey;
+    }
+
+    LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
+        (last ? ", last=true" : "") + ")");
+
+    if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
+      LOG.info("There is no contents");
+      return null;
+    }
+
+    if (comparator.compare(end, idxReader.getFirstKey()) < 0 ||
+        comparator.compare(idxReader.getLastKey(), start) < 0) {
+      LOG.info("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
+          "], but request start:" + start + ", end: " + end);
+      return null;
+    }
+
+    long startOffset;
+    long endOffset;
+    try {
+      startOffset = idxReader.find(start);
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + new TupleRange(keySchema, start, end) + ", idx min: "
+          + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+    try {
+      endOffset = idxReader.find(end);
+      if (endOffset == -1) {
+        endOffset = idxReader.find(end, true);
+      }
+    } catch (IOException ioe) {
+      LOG.error("State Dump (the requested range: "
+          + new TupleRange(keySchema, start, end) + ", idx min: "
+          + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+      throw ioe;
+    }
+
+    // if startOffset == -1 then case 2-1 or case 3
+    if (startOffset == -1) { // this is a hack
+      // if case 2-1 or case 3
+      try {
+        startOffset = idxReader.find(start, true);
+      } catch (IOException ioe) {
+        LOG.error("State Dump (the requested range: "
+            + new TupleRange(keySchema, start, end) + ", idx min: "
+            + idxReader.getFirstKey() + ", idx max: "
+            + idxReader.getLastKey());
+        throw ioe;
+      }
+    }
+
+    if (startOffset == -1) {
+      throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
+          "State Dump (the requested range: "
+          + new TupleRange(keySchema, start, end) + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+          + idxReader.getLastKey());
+    }
+
+    // if greater than indexed values
+    if (last || (endOffset == -1
+        && comparator.compare(idxReader.getLastKey(), end) < 0)) {
+      endOffset = data.length();
+    }
+
+    FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
+    LOG.info("Retrieve File Chunk: " + chunk);
+    return chunk;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
index 1fe5a0f..67e7423 100644
--- a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
@@ -22,14 +22,14 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.pullserver.FileAccessForbiddenException;
 import org.apache.tajo.util.TajoIdUtils;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -75,13 +75,14 @@ public class AdvancedDataRetriever implements DataRetriever {
 
     if (params.containsKey("sid")) {
       List<FileChunk> chunks = Lists.newArrayList();
-      List<String> qids = splitMaps(params.get("qid"));
-      for (String qid : qids) {
-        String[] ids = qid.split("_");
-        SubQueryId suid = TajoIdUtils.newSubQueryId(params.get("sid").get(0));
-        QueryUnitId quid = new QueryUnitId(suid, Integer.parseInt(ids[0]));
-        QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid,
-            Integer.parseInt(ids[1]));
+      List<String> queryUnidIds = splitMaps(params.get("qid"));
+      for (String eachQueryUnitId : queryUnidIds) {
+        String[] queryUnitIdSeqTokens = eachQueryUnitId.split("_");
+        ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
+        QueryUnitId quid = new QueryUnitId(ebId, Integer.parseInt(queryUnitIdSeqTokens[0]));
+
+        QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid, Integer.parseInt(queryUnitIdSeqTokens[1]));
+
         RetrieverHandler handler = handlerMap.get(attemptId.toString());
         FileChunk chunk = handler.get(params);
         chunks.add(chunk);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
index 0f6ed41..0896bd1 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -53,7 +53,7 @@ public class CSVFile {
     private TableStatistics stats = null;
 
     public CSVAppender(Configuration conf, final TableMeta meta,
-        final Path path) throws IOException {
+                       final Path path) throws IOException {
       super(conf, meta, path);
       this.fs = path.getFileSystem(conf);
       this.meta = meta;
@@ -94,60 +94,60 @@ public class CSVFile {
         } else {
           col = schema.getColumn(i);
           switch (col.getDataType().getType()) {
-          case BOOLEAN:
-            sb.append(tuple.getBoolean(i));
-            break;
-          case BIT:
-            sb.append(new String(Base64.encodeBase64(tuple.getByte(i)
-                .asByteArray(), false)));
-            break;
-          case BLOB:
-            sb.append(new String(Base64.encodeBase64(tuple.getBytes(i)
-                .asByteArray(), false)));
-            break;
-          case CHAR:
-            sb.append(tuple.getChar(i));
-            break;
+            case BOOLEAN:
+              sb.append(tuple.getBoolean(i));
+              break;
+            case BIT:
+              sb.append(new String(Base64.encodeBase64(tuple.getByte(i)
+                  .asByteArray(), false)));
+              break;
+            case BLOB:
+              sb.append(new String(Base64.encodeBase64(tuple.getBytes(i)
+                  .asByteArray(), false)));
+              break;
+            case CHAR:
+              sb.append(tuple.getChar(i));
+              break;
 //          case STRING:
 //            sb.append(tuple.getString(i));
 //            break;
-          case TEXT:
-            TextDatum td = tuple.getText(i);
-            sb.append(td.toString());
-            break;
-          case INT2:
-            sb.append(tuple.getShort(i));
-            break;
-          case INT4:
-            sb.append(tuple.getInt(i));
-            break;
-          case INT8:
-            sb.append(tuple.getLong(i));
-            break;
-          case FLOAT4:
-            sb.append(tuple.getFloat(i));
-            break;
-          case FLOAT8:
-            sb.append(tuple.getDouble(i));
-            break;
-          case INET4:
-            sb.append(tuple.getIPv4(i));
-            break;
-          case INET6:
-            sb.append(tuple.getIPv6(i));
-          case ARRAY:
+            case TEXT:
+              TextDatum td = tuple.getText(i);
+              sb.append(td.toString());
+              break;
+            case INT2:
+              sb.append(tuple.getShort(i));
+              break;
+            case INT4:
+              sb.append(tuple.getInt(i));
+              break;
+            case INT8:
+              sb.append(tuple.getLong(i));
+              break;
+            case FLOAT4:
+              sb.append(tuple.getFloat(i));
+              break;
+            case FLOAT8:
+              sb.append(tuple.getDouble(i));
+              break;
+            case INET4:
+              sb.append(tuple.getIPv4(i));
+              break;
+            case INET6:
+              sb.append(tuple.getIPv6(i));
+            case ARRAY:
             /*
              * sb.append("["); boolean first = true; ArrayDatum array =
              * (ArrayDatum) tuple.get(i); for (Datum field : array.toArray()) {
              * if (first) { first = false; } else { sb.append(delimiter); }
              * sb.append(field.asChars()); } sb.append("]");
              */
-            ArrayDatum array = (ArrayDatum) tuple.get(i);
-            sb.append(array.toJson());
-            break;
-          default:
-            throw new UnsupportedOperationException("Cannot write such field: "
-                + tuple.get(i).type());
+              ArrayDatum array = (ArrayDatum) tuple.get(i);
+              sb.append(array.toJson());
+              break;
+            default:
+              throw new UnsupportedOperationException("Cannot write such field: "
+                  + tuple.get(i).type());
           }
         }
         sb.append(delimiter);
@@ -195,7 +195,7 @@ public class CSVFile {
 
   public static class CSVScanner extends FileScanner implements SeekableScanner {
     public CSVScanner(Configuration conf, final TableMeta meta,
-        final Fragment fragment) throws IOException {
+                      final Fragment fragment) throws IOException {
       super(conf, meta, fragment);
     }
 
@@ -240,6 +240,11 @@ public class CSVFile {
       }
       super.init();
 
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("CSVScanner open:" + fragment.getPath() + "," + startOffset + "," + length +
+            "," + fs.getFileStatus(fragment.getPath()).getLen());
+      }
+
       if (startOffset != 0) {
         fis.seek(startOffset - 1);
         while (fis.readByte() != LF) {
@@ -297,7 +302,6 @@ public class CSVFile {
           while ((temp[cnt] = fis.readByte()) != LF) {
             cnt++;
           }
-
           // Replace tuple
           tuples[tuples.length - 1] = tuples[tuples.length - 1] + new String(temp, 0, cnt);
           validIdx = tuples.length;
@@ -319,7 +323,6 @@ public class CSVFile {
         this.tupleOffsets[i] = curTupleOffset + this.pageStart;
         curTupleOffset += this.tuples[i].getBytes().length + 1;//tuple byte +  1byte line feed
       }
-      
     }
 
     @Override
@@ -351,44 +354,44 @@ public class CSVFile {
               tuple.put(tid, DatumFactory.createNullDatum());
             } else {
               switch (field.getDataType().getType()) {
-              case BOOLEAN:
-                tuple.put(tid, DatumFactory.createBool(cell));
-                break;
-              case BIT:
-                tuple.put(tid, DatumFactory.createBit(Base64.decodeBase64(cell)[0]));
-                break;
-              case CHAR:
-                tuple.put(tid, DatumFactory.createChar(cell.charAt(0)));
-                break;
-              case BLOB:
-                tuple.put(tid, DatumFactory.createBlob(Base64.decodeBase64(cell)));
-                break;
-              case INT2:
-                tuple.put(tid, DatumFactory.createInt2(cell));
-                break;
-              case INT4:
-                tuple.put(tid, DatumFactory.createInt4(cell));
-                break;
-              case INT8:
-                tuple.put(tid, DatumFactory.createInt8(cell));
-                break;
-              case FLOAT4:
-                tuple.put(tid, DatumFactory.createFloat4(cell));
-                break;
-              case FLOAT8:
-                tuple.put(tid, DatumFactory.createFloat8(cell));
-                break;
-              case TEXT:
-                tuple.put(tid, DatumFactory.createText(cell));
-                break;
-              case INET4:
-                tuple.put(tid, DatumFactory.createInet4(cell));
-                break;
-              case ARRAY:
-                Datum data = StorageGsonHelper.getInstance().fromJson(cell,
-                    Datum.class);
-                tuple.put(tid, data);
-                break;
+                case BOOLEAN:
+                  tuple.put(tid, DatumFactory.createBool(cell));
+                  break;
+                case BIT:
+                  tuple.put(tid, DatumFactory.createBit(Base64.decodeBase64(cell)[0]));
+                  break;
+                case CHAR:
+                  tuple.put(tid, DatumFactory.createChar(cell.charAt(0)));
+                  break;
+                case BLOB:
+                  tuple.put(tid, DatumFactory.createBlob(Base64.decodeBase64(cell)));
+                  break;
+                case INT2:
+                  tuple.put(tid, DatumFactory.createInt2(cell));
+                  break;
+                case INT4:
+                  tuple.put(tid, DatumFactory.createInt4(cell));
+                  break;
+                case INT8:
+                  tuple.put(tid, DatumFactory.createInt8(cell));
+                  break;
+                case FLOAT4:
+                  tuple.put(tid, DatumFactory.createFloat4(cell));
+                  break;
+                case FLOAT8:
+                  tuple.put(tid, DatumFactory.createFloat8(cell));
+                  break;
+                case TEXT:
+                  tuple.put(tid, DatumFactory.createText(cell));
+                  break;
+                case INET4:
+                  tuple.put(tid, DatumFactory.createInet4(cell));
+                  break;
+                case ARRAY:
+                  Datum data = StorageGsonHelper.getInstance().fromJson(cell,
+                      Datum.class);
+                  tuple.put(tid, data);
+                  break;
               }
             }
           }
@@ -430,7 +433,7 @@ public class CSVFile {
       int tupleIndex = Arrays.binarySearch(this.tupleOffsets, offset);
       if (tupleIndex > -1) {
         this.currentIdx = tupleIndex;
-      } else if (offset >= this.pageStart + this.bufSize 
+      } else if (offset >= this.pageStart + this.bufSize
           + this.prevTailLen - this.tail.length || offset <= this.pageStart) {
         fis.seek(offset);
         tail = new byte[0];
@@ -441,10 +444,10 @@ public class CSVFile {
         // pageBuffer();
       } else {
         throw new IOException("invalid offset " +
-           " < pageStart : " +  this.pageStart + " , " + 
-           "  pagelength : " + this.bufSize + " , " + 
-           "  tail lenght : " + this.tail.length +
-           "  input offset : " + offset + " >");
+            " < pageStart : " +  this.pageStart + " , " +
+            "  pagelength : " + this.bufSize + " , " +
+            "  tail lenght : " + this.tail.length +
+            "  input offset : " + offset + " >");
       }
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
index 8b7ec67..a7a1e4a 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
@@ -80,9 +80,13 @@ public class MergeScanner implements Scanner {
 
   @Override
   public void close() throws IOException {
-    currentScanner.close();
+    if(currentScanner != null) {
+      currentScanner.close();
+    }
     iterator = null;
-    fragments.clear();
+    if(fragments != null) {
+      fragments.clear();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/start-tajo.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/start-tajo.sh b/tajo-dist/src/main/bin/start-tajo.sh
index a39ebf1..3d2ad6a 100755
--- a/tajo-dist/src/main/bin/start-tajo.sh
+++ b/tajo-dist/src/main/bin/start-tajo.sh
@@ -26,3 +26,11 @@ bin=`cd "$bin"; pwd`
 
 # start the tajo master daemon
 "$bin"/tajo-daemon.sh --config $TAJO_CONF_DIR start master
+
+if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
+  . "${TAJO_CONF_DIR}/tajo-env.sh"
+fi
+
+if [ "$TAJO_WORKER_STANDBY_MODE" = "true" ]; then
+  exec "$bin/tajo-daemons.sh" cd "$TAJO_HOME" \; "$bin/tajo-daemon.sh" start worker
+fi

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/stop-tajo.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/stop-tajo.sh b/tajo-dist/src/main/bin/stop-tajo.sh
index d22ca4a..88587ae 100755
--- a/tajo-dist/src/main/bin/stop-tajo.sh
+++ b/tajo-dist/src/main/bin/stop-tajo.sh
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 
-# Stop tajo map reduce daemons.  Run this on master node.
+# Stop tajo master daemons.  Run this on master node.
 
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
@@ -25,3 +25,12 @@ bin=`cd "$bin"; pwd`
 . "$bin"/tajo-config.sh
 
 "$bin"/tajo-daemon.sh --config $TAJO_CONF_DIR stop master
+
+if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
+  . "${TAJO_CONF_DIR}/tajo-env.sh"
+fi
+
+if [ "$TAJO_WORKER_STANDBY_MODE" = "true" ]; then
+  exec "$bin/tajo-daemons.sh" cd "$TAJO_HOME" \; "$bin/tajo-daemon.sh" stop worker
+fi
+

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo
index a67fe2b..5d5774e 100755
--- a/tajo-dist/src/main/bin/tajo
+++ b/tajo-dist/src/main/bin/tajo
@@ -61,6 +61,7 @@ if [ $# = 0 ]; then
   echo "Usage: tajo [--config confdir] COMMAND"
   echo "where COMMAND is one of:"
   echo "  master               run the Master Server"
+  echo "  worker               run the Worker Server"
   echo "  catalog              run the Catalog server"
   echo "  catutil              catalog utility"
   echo "  cli                  run the tajo cli"
@@ -272,6 +273,9 @@ if [ "$COMMAND" = "classpath" ] ; then
 elif [ "$COMMAND" = "master" ] ; then
   CLASS='org.apache.tajo.master.TajoMaster'
   TAJO_OPTS="$TAJO_OPTS $TAJO_MASTER_OPTS"
+elif [ "$COMMAND" = "worker" ] ; then
+  CLASS='org.apache.tajo.worker.TajoWorker'
+  TAJO_OPTS="$TAJO_OPTS $TAJO_WORKER_OPTS"
 elif [ "$COMMAND" = "catalog" ] ; then
   CLASS='org.apache.tajo.catalog.CatalogServer'
   TAJO_OPTS="$TAJO_OPTS $TAJO_CATALOG_OPTS"

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo-config.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo-config.sh b/tajo-dist/src/main/bin/tajo-config.sh
index 36c20d8..5972f46 100755
--- a/tajo-dist/src/main/bin/tajo-config.sh
+++ b/tajo-dist/src/main/bin/tajo-config.sh
@@ -53,4 +53,17 @@ then
 fi
  
 # Allow alternate conf dir location.
-TAJO_CONF_DIR="${TAJO_CONF_DIR:-$TAJO_HOME/conf}"
\ No newline at end of file
+TAJO_CONF_DIR="${TAJO_CONF_DIR:-$TAJO_HOME/conf}"
+
+#check to see it is specified whether to use the workers or the
+# workers file
+if [ $# -gt 1 ]
+then
+    if [ "--hosts" = "$1" ]
+    then
+        shift
+        workesfile=$1
+        shift
+        export TAJO_WORKERS="${TAJO_CONF_DIR}/$workesfile"
+    fi
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo-daemon.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo-daemon.sh b/tajo-dist/src/main/bin/tajo-daemon.sh
index 4a86c01..1d4a22b 100755
--- a/tajo-dist/src/main/bin/tajo-daemon.sh
+++ b/tajo-dist/src/main/bin/tajo-daemon.sh
@@ -17,7 +17,7 @@
 # limitations under the License.
 
 
-# Runs a Hadoop command as a daemon.
+# Runs a Tajo command as a daemon.
 #
 # Environment Variables
 #
@@ -69,11 +69,11 @@ if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
   . "${TAJO_CONF_DIR}/tajo-env.sh"
 fi
 
-# Determine if we're starting a secure datanode, and if so, redefine appropriate variables
-if [ "$command" == "datanode" ] && [ "$EUID" -eq 0 ] && [ -n "$TAJO_SECURE_DN_USER" ]; then
-  export TAJO_PID_DIR=$TAJO_SECURE_DN_PID_DIR
-  export TAJO_LOG_DIR=$TAJO_SECURE_DN_LOG_DIR
-  export TAJO_IDENT_STRING=$TAJO_SECURE_DN_USER   
+# Determine if we're starting a secure server, and if so, redefine appropriate variables
+if [ "$command" == "worker" ] && [ "$EUID" -eq 0 ] && [ -n "$TAJO_SECURE_WORKER_USER" ]; then
+  export TAJO_PID_DIR=$TAJO_SECURE_WORKER_PID_DIR
+  export TAJO_LOG_DIR=$TAJO_SECURE_WORKER_LOG_DIR
+  export TAJO_IDENT_STRING=$TAJO_SECURE_WORKER_USER
 fi
 
 if [ "$TAJO_IDENT_STRING" = "" ]; then

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo-daemons.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo-daemons.sh b/tajo-dist/src/main/bin/tajo-daemons.sh
new file mode 100644
index 0000000..74c599f
--- /dev/null
+++ b/tajo-dist/src/main/bin/tajo-daemons.sh
@@ -0,0 +1,68 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Run a shell command on all worker hosts.
+#
+# Environment Variables
+#
+#   TAJO_WORKERS    File naming remote hosts.
+#     Default is ${TAJO_CONF_DIR}/workers.
+#   TAJO_CONF_DIR  Alternate conf dir. Default is ${TAJO_HOME}/conf.
+#   TAJO_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
+#   TAJO_SSH_OPTS Options passed to ssh when running remote commands.
+##
+
+usage="Usage: tajo-daemons.sh command..."
+
+# if no args specified, show usage
+if [ $# -le 0 ]; then
+  echo $usage
+  exit 1
+fi
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/tajo-config.sh
+
+# If the workers file is specified in the command line,
+# then it takes precedence over the definition in
+# neptune-env.sh. Save it here.
+HOSTLIST=$TAJO_WORKERS
+
+if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
+  . "${TAJO_CONF_DIR}/tajo-env.sh"
+fi
+
+if [ "$HOSTLIST" = "" ]; then
+  if [ "$TAJO_WORKERS" = "" ]; then
+    export HOSTLIST="${TAJO_CONF_DIR}/workers"
+  else
+    export HOSTLIST="${TAJO_CONF_DIR}"
+  fi
+fi
+
+for slave in `cat "$HOSTLIST"`; do
+ ssh $TAJO_SSH_OPTS $slave $"${@// /\\ }" \
+   2>&1 | sed "s/^/$slave: /" &
+ if [ "$TAJO_SLAVE_SLEEP" != "" ]; then
+   sleep $TAJO_SLAVE_SLEEP
+ fi
+done
+
+wait

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/conf/tajo-env.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/conf/tajo-env.sh b/tajo-dist/src/main/conf/tajo-env.sh
index 3996030..bca6d2b 100755
--- a/tajo-dist/src/main/conf/tajo-env.sh
+++ b/tajo-dist/src/main/conf/tajo-env.sh
@@ -34,14 +34,23 @@
 # export TAJO_OPTS=-server
 export TAJO_OPTS=-XX:+PrintGCTimeStamps
 
+# Extra TajoMaster's java runtime options for TajoMaster. Empty by default
+# export TAJO_MASTER_OPTS=
+
+# Extra TajoWorker's java runtime options for TajoMaster. Empty by default
+# export TAJO_WORKER_OPTS=
+
 # Where log files are stored.  $TAJO_HOME/logs by default.
 # export TAJO_LOG_DIR=${TAJO_HOME}/logs
 
 # The directory where pid files are stored. /tmp by default.
-# export TAJO_PID_DIR=/var/hadoop/pids
+# export TAJO_PID_DIR=/var/tajo/pids
 
-# A string representing this instance of hadoop. $USER by default.
+# A string representing this instance of tajo. $USER by default.
 # export TAJO_IDENT_STRING=$USER
 
 # The scheduling priority for daemon processes.  See 'man nice'.
 # export TAJO_NICENESS=10
+
+export TAJO_WORKER_STANDBY_MODE=true
+

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/conf/workers
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/conf/workers b/tajo-dist/src/main/conf/workers
new file mode 100644
index 0000000..d18580b
--- /dev/null
+++ b/tajo-dist/src/main/conf/workers
@@ -0,0 +1 @@
+localhost
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
index b520b3e..154fb9b 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
@@ -143,7 +143,6 @@ public class NettyServerBase {
         }
       }
       if (available(port)) {
-        LOG.info("Detect an unused port:" + port);
         return port;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
index c58db58..d78e4e1 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
@@ -22,10 +22,10 @@ import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.*;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.*;
 import org.apache.tajo.rpc.RpcProtos.RpcRequest;
 import org.apache.tajo.rpc.RpcProtos.RpcResponse;
 import org.apache.tajo.util.NetUtils;
+import org.jboss.netty.channel.*;
 
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
@@ -198,8 +198,9 @@ public class ProtoAsyncRpcClient extends NettyClientBase {
     @Override
     public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
         throws Exception {
+      LOG.error(addr + "," + protocol + "," + e.getCause().getMessage(), e.getCause());
       e.getChannel().close();
-      throw new RemoteException(getErrorMessage(""), e.getCause());
+      throw new RemoteException(getErrorMessage(addr.toString()), e.getCause());
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
index 56faaba..1a6ce29 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
@@ -25,9 +25,9 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.*;
 import org.apache.tajo.rpc.RpcProtos.RpcRequest;
 import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+import org.jboss.netty.channel.*;
 
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
@@ -118,7 +118,6 @@ public class ProtoAsyncRpcServer extends NettyServerBase {
         RemoteCallException callException = (RemoteCallException) e.getCause();
         e.getChannel().write(callException.getResponse());
       }
-
       throw new RemoteException(e.getCause());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
index 2018f6d..2a0b01f 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
@@ -25,10 +25,10 @@ import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.*;
 import org.apache.tajo.rpc.RpcProtos.RpcRequest;
 import org.apache.tajo.rpc.RpcProtos.RpcResponse;
 import org.apache.tajo.util.NetUtils;
+import org.jboss.netty.channel.*;
 
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
@@ -129,9 +129,13 @@ public class ProtoBlockingRpcClient extends NettyClientBase {
   }
 
   private String getErrorMessage(String message) {
-    return "Exception [" + protocol.getCanonicalName() +
-        "(" + NetUtils.normalizeInetSocketAddress((InetSocketAddress)
-        getChannel().getRemoteAddress()) + ")]: " + message;
+    if(protocol != null && getChannel() != null) {
+      return "Exception [" + protocol.getCanonicalName() +
+          "(" + NetUtils.normalizeInetSocketAddress((InetSocketAddress)
+          getChannel().getRemoteAddress()) + ")]: " + message;
+    } else {
+      return "Exception " + message;
+    }
   }
 
   private class ClientChannelUpstreamHandler extends SimpleChannelUpstreamHandler {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java b/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
index 6dff443..6a43be3 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
@@ -99,4 +99,4 @@ public class NetUtils {
     }
     return host;
   }
-}
+}
\ No newline at end of file


[7/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
index 7ae01d6..044f0ae 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
@@ -21,417 +21,52 @@ package org.apache.tajo.master;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.ContainerManager;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.util.ProtoUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.QueryConf;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.querymaster.QueryMaster;
-import org.apache.tajo.pullserver.PullServerAuxService;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.util.*;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
 
 public abstract class ContainerProxy {
-  private static final Log LOG = LogFactory.getLog(ContainerProxy.class);
+  protected static final Log LOG = LogFactory.getLog(ContainerProxy.class);
 
   final public static FsPermission QUERYCONF_FILE_PERMISSION =
           FsPermission.createImmutable((short) 0644); // rw-r--r--
 
-  private final static RecordFactory recordFactory =
-          RecordFactoryProvider.getRecordFactory(null);
 
-  private static enum ContainerState {
+  protected static enum ContainerState {
     PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
   }
 
-  private final YarnRPC yarnRPC;
-  private Configuration conf;
-  private QueryMaster.QueryContext context;
+  protected final ExecutionBlockId executionBlockId;
+  protected Configuration conf;
+  protected QueryMasterTask.QueryContext context;
 
-  private ContainerState state;
+  protected ContainerState state;
   // store enough information to be able to cleanup the container
-  private Container container;
-  private ContainerId containerID;
-  final private String containerMgrAddress;
-  private ContainerToken containerToken;
-  private String hostName;
-  private int port = -1;
+  protected Container container;
+  protected ContainerId containerID;
+  protected String hostName;
+  protected int port = -1;
 
-  protected abstract void containerStarted();
-  protected abstract String getId();
-  protected abstract String getRunnerClass();
-  protected abstract Vector<CharSequence> getTaskParams();
+  public abstract void launch(ContainerLaunchContext containerLaunchContext);
+  public abstract void stopContainer();
 
-  public ContainerProxy(QueryMaster.QueryContext context, Configuration conf, YarnRPC yarnRPC, Container container) {
+  public ContainerProxy(QueryMasterTask.QueryContext context, Configuration conf,
+                        ExecutionBlockId executionBlockId, Container container) {
     this.context = context;
     this.conf = conf;
-    this.yarnRPC = yarnRPC;
     this.state = ContainerState.PREP;
     this.container = container;
+    this.executionBlockId = executionBlockId;
     this.containerID = container.getId();
-    NodeId nodeId = container.getNodeId();
-    this.containerMgrAddress = nodeId.getHost() + ":" + nodeId.getPort();
-    this.containerToken = container.getContainerToken();
-  }
-
-  protected ContainerManager getCMProxy(ContainerId containerID,
-                                        final String containerManagerBindAddr,
-                                        ContainerToken containerToken)
-          throws IOException {
-    String [] hosts = containerManagerBindAddr.split(":");
-    final InetSocketAddress cmAddr =
-            new InetSocketAddress(hosts[0], Integer.parseInt(hosts[1]));
-    UserGroupInformation user = UserGroupInformation.getCurrentUser();
-
-    if (UserGroupInformation.isSecurityEnabled()) {
-      Token<ContainerTokenIdentifier> token =
-              ProtoUtils.convertFromProtoFormat(containerToken, cmAddr);
-      // the user in createRemoteUser in this context has to be ContainerID
-      user = UserGroupInformation.createRemoteUser(containerID.toString());
-      user.addToken(token);
-    }
-
-    ContainerManager proxy = user.doAs(new PrivilegedAction<ContainerManager>() {
-      @Override
-      public ContainerManager run() {
-        return (ContainerManager) yarnRPC.getProxy(ContainerManager.class, cmAddr, conf);
-      }
-    });
-
-    return proxy;
   }
 
   public synchronized boolean isCompletelyDone() {
     return state == ContainerState.DONE || state == ContainerState.FAILED;
   }
 
-  @SuppressWarnings("unchecked")
-  public synchronized void launch(ContainerLaunchContext commonContainerLaunchContext) {
-    LOG.info("Launching Container with Id: " + containerID);
-    if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
-      state = ContainerState.DONE;
-      LOG.error("Container (" + containerID + " was killed before it was launched");
-      return;
-    }
-
-    ContainerManager proxy = null;
-    try {
-
-      proxy = getCMProxy(containerID, containerMgrAddress,
-              containerToken);
-
-      // Construct the actual Container
-      ContainerLaunchContext containerLaunchContext = createContainerLaunchContext(commonContainerLaunchContext);
-
-      // Now launch the actual container
-      StartContainerRequest startRequest = Records
-              .newRecord(StartContainerRequest.class);
-      startRequest.setContainerLaunchContext(containerLaunchContext);
-      StartContainerResponse response = proxy.startContainer(startRequest);
-
-      ByteBuffer portInfo = response
-              .getServiceResponse(PullServerAuxService.PULLSERVER_SERVICEID);
-
-      if(portInfo != null) {
-        port = PullServerAuxService.deserializeMetaData(portInfo);
-      }
-
-      LOG.info("PullServer port returned by ContainerManager for "
-              + containerID + " : " + port);
-
-      if(port < 0) {
-        this.state = ContainerState.FAILED;
-        throw new IllegalStateException("Invalid shuffle port number "
-                + port + " returned for " + containerID);
-      }
-
-      containerStarted();
-
-      this.state = ContainerState.RUNNING;
-      this.hostName = containerMgrAddress.split(":")[0];
-      context.addContainer(containerID, this);
-    } catch (Throwable t) {
-      String message = "Container launch failed for " + containerID + " : "
-              + StringUtils.stringifyException(t);
-      this.state = ContainerState.FAILED;
-      LOG.error(message);
-    } finally {
-      if (proxy != null) {
-        yarnRPC.stopProxy(proxy, conf);
-      }
-    }
-  }
-
-  public synchronized void kill() {
-
-    if(isCompletelyDone()) {
-      return;
-    }
-    if(this.state == ContainerState.PREP) {
-      this.state = ContainerState.KILLED_BEFORE_LAUNCH;
-    } else {
-      LOG.info("KILLING " + containerID);
-
-      ContainerManager proxy = null;
-      try {
-        proxy = getCMProxy(this.containerID, this.containerMgrAddress,
-                this.containerToken);
-
-        // kill the remote container if already launched
-        StopContainerRequest stopRequest = Records
-                .newRecord(StopContainerRequest.class);
-        stopRequest.setContainerId(this.containerID);
-        proxy.stopContainer(stopRequest);
-        // If stopContainer returns without an error, assuming the stop made
-        // it over to the NodeManager.
-//          context.getEventHandler().handle(
-//              new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
-        context.removeContainer(containerID);
-      } catch (Throwable t) {
-
-        // ignore the cleanup failure
-        String message = "cleanup failed for container "
-                + this.containerID + " : "
-                + StringUtils.stringifyException(t);
-        LOG.warn(message);
-        this.state = ContainerState.DONE;
-        return;
-      } finally {
-        if (proxy != null) {
-          yarnRPC.stopProxy(proxy, conf);
-        }
-      }
-      this.state = ContainerState.DONE;
-    }
-  }
-
-  public static ContainerLaunchContext createCommonContainerLaunchContext(Configuration config, String queryId, boolean isMaster) {
-    TajoConf conf = (TajoConf)config;
-
-    ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
-    try {
-      ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-    ////////////////////////////////////////////////////////////////////////////
-    // Set the env variables to be setup
-    ////////////////////////////////////////////////////////////////////////////
-    LOG.info("Set the environment for the application master");
-
-    Map<String, String> environment = new HashMap<String, String>();
-    //String initialClassPath = getInitialClasspath(conf);
-    environment.put(ApplicationConstants.Environment.SHELL.name(), "/bin/bash");
-    if(System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()) != null) {
-      environment.put(ApplicationConstants.Environment.JAVA_HOME.name(),
-          System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()));
-    }
-
-    // TODO - to be improved with org.apache.tajo.sh shell script
-    Properties prop = System.getProperties();
-
-    if (prop.getProperty("tajo.test", "FALSE").equalsIgnoreCase("TRUE") ||
-            (System.getenv("tajo.test") != null && System.getenv("tajo.test").equalsIgnoreCase("TRUE"))) {
-      LOG.info("tajo.test is TRUE");
-      environment.put(ApplicationConstants.Environment.CLASSPATH.name(), prop.getProperty("java.class.path", null));
-      environment.put("tajo.test", "TRUE");
-    } else {
-      // Add AppMaster.jar location to classpath
-      // At some point we should not be required to add
-      // the hadoop specific classpaths to the env.
-      // It should be provided out of the box.
-      // For now setting all required classpaths including
-      // the classpath to "." for the application jar
-      StringBuilder classPathEnv = new StringBuilder("./");
-      //for (String c : conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
-      for (String c : YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH) {
-        classPathEnv.append(':');
-        classPathEnv.append(c.trim());
-      }
-
-      classPathEnv.append(":" + System.getenv("TAJO_BASE_CLASSPATH"));
-      classPathEnv.append(":./log4j.properties:./*");
-      if(System.getenv("HADOOP_HOME") != null) {
-        environment.put("HADOOP_HOME", System.getenv("HADOOP_HOME"));
-        environment.put(ApplicationConstants.Environment.HADOOP_COMMON_HOME.name(), System.getenv("HADOOP_HOME"));
-        environment.put(ApplicationConstants.Environment.HADOOP_HDFS_HOME.name(), System.getenv("HADOOP_HOME"));
-        environment.put(ApplicationConstants.Environment.HADOOP_YARN_HOME.name(), System.getenv("HADOOP_HOME"));
-      }
-
-      if(System.getenv("TAJO_BASE_CLASSPATH") != null) {
-        environment.put("TAJO_BASE_CLASSPATH", System.getenv("TAJO_BASE_CLASSPATH"));
-      }
-      environment.put(ApplicationConstants.Environment.CLASSPATH.name(), classPathEnv.toString());
-    }
-
-    ctx.setEnvironment(environment);
-
-    ////////////////////////////////////////////////////////////////////////////
-    // Set the local resources
-    ////////////////////////////////////////////////////////////////////////////
-    Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
-    FileSystem fs = null;
-
-    LOG.info("defaultFS: " + conf.get("fs.defaultFS"));
-
-    try {
-      fs = FileSystem.get(conf);
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
-    }
-
-    FileContext fsCtx = null;
-    try {
-      fsCtx = FileContext.getFileContext(conf);
-    } catch (UnsupportedFileSystemException e) {
-      LOG.error(e.getMessage(), e);
-    }
-
-
-    LOG.info("Writing a QueryConf to HDFS and add to local environment");
-    try {
-      // TODO move to tajo temp
-      Path warehousePath = new Path(conf.getVar(TajoConf.ConfVars.ROOT_DIR), TajoConstants.WAREHOUSE_DIR);
-      Path queryConfPath = new Path(warehousePath, queryId);
-      if(isMaster) {
-        queryConfPath = new Path(queryConfPath, QueryConf.QUERY_MASTER_FILENAME);
-      } else {
-        queryConfPath = new Path(queryConfPath, QueryConf.FILENAME);
-      }
-
-      if(!fs.exists(queryConfPath)){
-        writeConf(conf, queryConfPath);
-      } else {
-        LOG.warn("QueryConf already exist. path: "  + queryConfPath.toString());
-      }
-      LocalResource queryConfSrc = createApplicationResource(fsCtx, queryConfPath, LocalResourceType.FILE);
-
-      localResources.put(queryConfPath.getName(), queryConfSrc);
-      ctx.setLocalResources(localResources);
-
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
-    }
-
-    // TODO - move to sub-class
-    // Add shuffle token
-    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
-    try {
-      serviceData.put(PullServerAuxService.PULLSERVER_SERVICEID, PullServerAuxService.serializeMetaData(0));
-    } catch (IOException ioe) {
-      LOG.error(ioe);
-    }
-    ctx.setServiceData(serviceData);
-
-    return ctx;
-  }
-
-  private static LocalResource createApplicationResource(FileContext fs,
-                                                  Path p, LocalResourceType type)
-          throws IOException {
-    LocalResource rsrc = recordFactory.newRecordInstance(LocalResource.class);
-    FileStatus rsrcStat = fs.getFileStatus(p);
-    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs
-            .getDefaultFileSystem().resolvePath(rsrcStat.getPath())));
-    rsrc.setSize(rsrcStat.getLen());
-    rsrc.setTimestamp(rsrcStat.getModificationTime());
-    rsrc.setType(type);
-    rsrc.setVisibility(LocalResourceVisibility.APPLICATION);
-    return rsrc;
-  }
-
-  private static void writeConf(Configuration conf, Path queryConfFile)
-          throws IOException {
-    // Write job file to Tajo's fs
-    FileSystem fs = queryConfFile.getFileSystem(conf);
-    FSDataOutputStream out =
-            FileSystem.create(fs, queryConfFile,
-                    new FsPermission(QUERYCONF_FILE_PERMISSION));
-    try {
-      conf.writeXml(out);
-    } finally {
-      out.close();
-    }
-  }
-
-  public ContainerLaunchContext createContainerLaunchContext(ContainerLaunchContext commonContainerLaunchContext) {
-    // Setup environment by cloning from common env.
-    Map<String, String> env = commonContainerLaunchContext.getEnvironment();
-    Map<String, String> myEnv = new HashMap<String, String>(env.size());
-    myEnv.putAll(env);
-
-    // Duplicate the ByteBuffers for access by multiple containers.
-    Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
-    for (Map.Entry<String, ByteBuffer> entry : commonContainerLaunchContext.getServiceData().entrySet()) {
-      myServiceData.put(entry.getKey(), entry.getValue().duplicate());
-    }
-
-    ////////////////////////////////////////////////////////////////////////////
-    // Set the local resources
-    ////////////////////////////////////////////////////////////////////////////
-    // Set the necessary command to execute the application master
-    Vector<CharSequence> vargs = new Vector<CharSequence>(30);
-
-    // Set java executable command
-    //LOG.info("Setting up app master command");
-    vargs.add("${JAVA_HOME}" + "/bin/java");
-    // Set Xmx based on am memory size
-    vargs.add("-Xmx2000m");
-    // Set Remote Debugging
-    //if (!context.getQuery().getSubQuery(event.getSubQueryId()).isLeafQuery()) {
-    //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
-    //}
-    // Set class name
-    vargs.add(getRunnerClass());
-    vargs.add(getId()); // subqueryId
-    vargs.add(containerMgrAddress); // nodeId
-    vargs.add(containerID.toString()); // containerId
-    Vector<CharSequence> taskParams = getTaskParams();
-    if(taskParams != null) {
-      vargs.addAll(taskParams);
-    }
-
-    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
-    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
-
-    // Get final commmand
-    StringBuilder command = new StringBuilder();
-    for (CharSequence str : vargs) {
-      command.append(str).append(" ");
-    }
-
-    LOG.info("Completed setting up TaskRunner command " + command.toString());
-    List<String> commands = new ArrayList<String>();
-    commands.add(command.toString());
-
-    return BuilderUtils.newContainerLaunchContext(containerID, commonContainerLaunchContext.getUser(),
-            container.getResource(), commonContainerLaunchContext.getLocalResources(), myEnv, commands,
-            myServiceData, null, new HashMap<ApplicationAccessType, String>());
-  }
-
   public String getTaskHostName() {
     return this.hostName;
   }
@@ -439,4 +74,8 @@ public abstract class ContainerProxy {
   public int getTaskPort() {
     return this.port;
   }
+
+  public String getId() {
+    return executionBlockId.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
index 36327ff..a92ef75 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
@@ -15,7 +15,7 @@
 package org.apache.tajo.master;
 
 import com.google.common.base.Preconditions;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.engine.planner.logical.*;
 
@@ -40,7 +40,7 @@ public class ExecutionBlock {
     RANGE
   }
 
-  private SubQueryId subQueryId;
+  private ExecutionBlockId executionBlockId;
   private LogicalNode plan = null;
   private StoreTableNode store = null;
   private List<ScanNode> scanlist = new ArrayList<ScanNode>();
@@ -50,12 +50,12 @@ public class ExecutionBlock {
   private boolean hasJoinPlan;
   private boolean hasUnionPlan;
 
-  public ExecutionBlock(SubQueryId subQueryId) {
-    this.subQueryId = subQueryId;
+  public ExecutionBlock(ExecutionBlockId executionBlockId) {
+    this.executionBlockId = executionBlockId;
   }
 
-  public SubQueryId getId() {
-    return subQueryId;
+  public ExecutionBlockId getId() {
+    return executionBlockId;
   }
 
   public String getOutputName() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 8c3617e..ee4b98d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -23,16 +23,11 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.client.YarnClient;
-import org.apache.hadoop.yarn.client.YarnClientImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.QueryConf;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoProtos;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.catalog.CatalogService;
 import org.apache.tajo.catalog.CatalogUtil;
@@ -48,18 +43,19 @@ import org.apache.tajo.engine.exception.UnknownWorkerException;
 import org.apache.tajo.engine.parser.SQLAnalyzer;
 import org.apache.tajo.engine.planner.*;
 import org.apache.tajo.engine.planner.global.GlobalOptimizer;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.logical.CreateTableNode;
+import org.apache.tajo.engine.planner.logical.DropTableNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.querymaster.QueryMasterManager;
+import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.querymaster.QueryJobManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.TajoIdUtils;
 
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.EnumSet;
-import java.util.Set;
 
 @SuppressWarnings("unchecked")
 public class GlobalEngine extends AbstractService {
@@ -76,9 +72,6 @@ public class GlobalEngine extends AbstractService {
   private GlobalPlanner globalPlanner;
   private GlobalOptimizer globalOptimizer;
 
-  // Yarn
-  protected YarnClient yarnClient;
-
   public GlobalEngine(final MasterContext context)
       throws IOException {
     super(GlobalEngine.class.getName());
@@ -89,7 +82,6 @@ public class GlobalEngine extends AbstractService {
 
   public void start() {
     try  {
-      connectYarnClient();
       analyzer = new SQLAnalyzer();
       planner = new LogicalPlanner(context.getCatalog());
       optimizer = new LogicalOptimizer();
@@ -98,138 +90,60 @@ public class GlobalEngine extends AbstractService {
 
       globalOptimizer = new GlobalOptimizer();
     } catch (Throwable t) {
-      t.printStackTrace();
+      LOG.error(t.getMessage(), t);
     }
     super.start();
   }
 
   public void stop() {
     super.stop();
-    if (yarnClient != null) {
-      yarnClient.stop();
-    }
   }
 
-  public QueryId executeQuery(String tql)
+  public ClientProtos.GetQueryStatusResponse executeQuery(String sql)
       throws InterruptedException, IOException,
       NoSuchQueryIdException, IllegalQueryStatusException,
       UnknownWorkerException, EmptyClusterException {
 
-    long querySubmittionTime = context.getClock().getTime();
-    LOG.info("SQL: " + tql);
+    LOG.info("SQL: " + sql);
     // parse the query
-    Expr planningContext = analyzer.parse(tql);
+    Expr planningContext = analyzer.parse(sql);
     LogicalRootNode plan = (LogicalRootNode) createLogicalPlan(planningContext);
 
+    ClientProtos.GetQueryStatusResponse.Builder responseBuilder = ClientProtos.GetQueryStatusResponse.newBuilder();
+
     if (PlannerUtil.checkIfDDLPlan(plan)) {
       updateQuery(plan.getChild());
-      return TajoIdUtils.NullQueryId;
+
+      responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+      responseBuilder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
     } else {
-      GetNewApplicationResponse newApp = yarnClient.getNewApplication();
-      ApplicationId appId = newApp.getApplicationId();
-      QueryId queryId = TajoIdUtils.createQueryId(appId, 0);
-
-      LOG.info("Get AppId: " + appId + ", QueryId: " + queryId);
-      LOG.info("Setting up application submission context for ASM");
-
-      //request QueryMaster container
-      QueryConf queryConf = new QueryConf(context.getConf());
-      queryConf.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
-      // the output table is given by user
-      if (plan.getChild().getType() == NodeType.CREATE_TABLE) {
-        CreateTableNode createTableNode = (CreateTableNode) plan.getChild();
-        queryConf.setOutputTable(createTableNode.getTableName());
+      QueryJobManager queryJobManager = context.getQueryJobManager();
+      QueryInfo queryInfo = null;
+      try {
+        queryInfo = queryJobManager.createNewQueryJob(sql, plan);
+      } catch (Exception e) {
+        responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+        responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
+        responseBuilder.setState(TajoProtos.QueryState.QUERY_ERROR);
+        responseBuilder.setErrorMessage(StringUtils.stringifyException(e));
+
+        return responseBuilder.build();
       }
-      QueryMasterManager queryMasterManager = new QueryMasterManager(context, yarnClient, queryId, tql, plan, appId,
-              context.getClock(), querySubmittionTime);
-      queryMasterManager.init(queryConf);
-      queryMasterManager.start();
-      context.addQuery(queryId, queryMasterManager);
-
-      return queryId;
-    }
-  }
-
-  private ApplicationAttemptId submitQuery() throws YarnRemoteException {
-    GetNewApplicationResponse newApp = getNewApplication();
-    ApplicationId appId = newApp.getApplicationId();
-    LOG.info("Get AppId: " + appId);
-    LOG.info("Setting up application submission context for ASM");
 
-    ApplicationSubmissionContext appContext = Records
-            .newRecord(ApplicationSubmissionContext.class);
+      //queryJobManager.getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInfo));
 
-    // set the application id
-    appContext.setApplicationId(appId);
-    // set the application name
-    appContext.setApplicationName("Tajo");
-
-    org.apache.hadoop.yarn.api.records.Priority
-            pri = Records.newRecord(org.apache.hadoop.yarn.api.records.Priority.class);
-    pri.setPriority(5);
-    appContext.setPriority(pri);
-
-    // Set the queue to which this application is to be submitted in the RM
-    appContext.setQueue("default");
-
-    ContainerLaunchContext amContainer = Records
-            .newRecord(ContainerLaunchContext.class);
-    appContext.setAMContainerSpec(amContainer);
-
-    LOG.info("Submitting application to ASM");
-    yarnClient.submitApplication(appContext);
-
-    ApplicationReport appReport = monitorApplication(appId,
-            EnumSet.of(YarnApplicationState.ACCEPTED));
-    ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
-    LOG.info("Launching application with id: " + attemptId);
-
-    return attemptId;
-  }
+      responseBuilder.setQueryId(queryInfo.getQueryId().getProto());
+      responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+      responseBuilder.setState(queryInfo.getQueryState());
+      if(queryInfo.getQueryMasterHost() != null) {
+        responseBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+      }
+      responseBuilder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
+    }
 
-  private ApplicationAttemptId submitQueryOld() throws YarnRemoteException {
-    GetNewApplicationResponse newApp = getNewApplication();
-    // Get a new application id
-    ApplicationId appId = newApp.getApplicationId();
-    LOG.info("Get AppId: " + appId);
-    LOG.info("Setting up application submission context for ASM");
-    ApplicationSubmissionContext appContext = Records
-        .newRecord(ApplicationSubmissionContext.class);
-
-    // set the application id
-    appContext.setApplicationId(appId);
-    // set the application name
-    appContext.setApplicationName("Tajo");
-
-    // Set the priority for the application master
-    org.apache.hadoop.yarn.api.records.Priority
-        pri = Records.newRecord(org.apache.hadoop.yarn.api.records.Priority.class);
-    pri.setPriority(5);
-    appContext.setPriority(pri);
-
-    // Set the queue to which this application is to be submitted in the RM
-    appContext.setQueue("default");
-
-    // Set up the container launch context for the application master
-    ContainerLaunchContext amContainer = Records
-        .newRecord(ContainerLaunchContext.class);
-    appContext.setAMContainerSpec(amContainer);
-
-    // unmanaged AM
-    appContext.setUnmanagedAM(true);
-    LOG.info("Setting unmanaged AM");
-
-    // Submit the application to the applications manager
-    LOG.info("Submitting application to ASM");
-    yarnClient.submitApplication(appContext);
-
-    // Monitor the application to wait for launch state
-    ApplicationReport appReport = monitorApplication(appId,
-        EnumSet.of(YarnApplicationState.ACCEPTED));
-    ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
-    LOG.info("Launching application with id: " + attemptId);
-
-    return attemptId;
+    ClientProtos.GetQueryStatusResponse response = responseBuilder.build();
+    return response;
   }
 
   public QueryId updateQuery(String sql) throws IOException, SQLException {
@@ -242,7 +156,7 @@ public class GlobalEngine extends AbstractService {
       throw new SQLException("This is not update query:\n" + sql);
     } else {
       updateQuery(plan.getChild());
-      return TajoIdUtils.NullQueryId;
+      return QueryIdFactory.NULL_QUERY_ID;
     }
   }
 
@@ -272,24 +186,14 @@ public class GlobalEngine extends AbstractService {
     } catch (PlanningException e) {
       LOG.error(e.getMessage(), e);
     }
-    LOG.info("LogicalPlan:\n" + plan.getRootBlock().getRoot());
 
-    return optimizedPlan;
-  }
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("LogicalPlan:\n" + plan.getRootBlock().getRoot());
+    }
 
-  private MasterPlan createGlobalPlan(QueryId id, LogicalRootNode rootNode)
-      throws IOException {
-    MasterPlan globalPlan = globalPlanner.build(id, rootNode);
-    return globalOptimizer.optimize(globalPlan);
+    return optimizedPlan;
   }
 
-//  private void startQuery(final QueryId queryId, final QueryConf queryConf,
-//                          final QueryMaster query) {
-//    context.getAllQueries().put(queryId, query);
-//    query.init(queryConf);
-//    query.start();
-//  }
-
   private TableDesc createTable(CreateTableNode createTable) throws IOException {
     TableMeta meta;
 
@@ -368,58 +272,4 @@ public class GlobalEngine extends AbstractService {
 
     LOG.info("Table \"" + tableName + "\" is dropped.");
   }
-
-  private void connectYarnClient() {
-    this.yarnClient = new YarnClientImpl();
-    this.yarnClient.init(context.getConf());
-    this.yarnClient.start();
-  }
-
-  public GetNewApplicationResponse getNewApplication()
-      throws YarnRemoteException {
-    return yarnClient.getNewApplication();
-  }
-
-  /**
-   * Monitor the submitted application for completion. Kill application if time
-   * expires.
-   *
-   * @param appId
-   *          Application Id of application to be monitored
-   * @return true if application completed successfully
-   * @throws YarnRemoteException
-   */
-  private ApplicationReport monitorApplication(ApplicationId appId,
-                                               Set<YarnApplicationState> finalState) throws YarnRemoteException {
-
-    while (true) {
-
-      // Check app status every 1 second.
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        LOG.debug("Thread sleep in monitoring loop interrupted");
-      }
-
-      // Get application report for the appId we are interested in
-      ApplicationReport report = yarnClient.getApplicationReport(appId);
-
-      LOG.info("Got application report from ASM for" + ", appId="
-          + appId.getId() + ", appAttemptId="
-          + report.getCurrentApplicationAttemptId() + ", clientToken="
-          + report.getClientToken() + ", appDiagnostics="
-          + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
-          + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
-          + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
-          + ", yarnAppState=" + report.getYarnApplicationState().toString()
-          + ", distributedFinalState="
-          + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
-          + report.getTrackingUrl() + ", appUser=" + report.getUser());
-
-      YarnApplicationState state = report.getYarnApplicationState();
-      if (finalState.contains(state)) {
-        return report;
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
index 8b6ba94..14c8bfd 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
@@ -21,9 +21,9 @@ package org.apache.tajo.master;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.conf.TajoConf;
@@ -75,9 +75,9 @@ public class GlobalPlanner {
     // insert store at the subnode of the root
     UnaryNode root = rootNode;
     if (root.getChild().getType() != NodeType.STORE) {
-      SubQueryId subQueryId = QueryIdFactory.newSubQueryId(this.queryId);
-      outputTableName = subQueryId.toString();
-      insertStore(subQueryId.toString(),root).setLocal(false);
+      ExecutionBlockId executionBlockId = QueryIdFactory.newExecutionBlockId(this.queryId);
+      outputTableName = executionBlockId.toString();
+      insertStore(executionBlockId.toString(),root).setLocal(false);
     }
     
     // convert 2-phase plan
@@ -113,10 +113,10 @@ public class GlobalPlanner {
         if (groupby.getChild().getType() != NodeType.UNION &&
             groupby.getChild().getType() != NodeType.STORE &&
             groupby.getChild().getType() != NodeType.SCAN) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           insertStore(tableId, groupby);
         }
-        tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+        tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
         // insert (a store for the first group by) and (a second group by)
         PlannerUtil.transformGroupbyTo2PWithStore((GroupbyNode)node, tableId);
       } else if (node.getType() == NodeType.SORT) {
@@ -126,10 +126,10 @@ public class GlobalPlanner {
         if (sort.getChild().getType() != NodeType.UNION &&
             sort.getChild().getType() != NodeType.STORE &&
             sort.getChild().getType() != NodeType.SCAN) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           insertStore(tableId, sort);
         }
-        tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+        tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
         // insert (a store for the first sort) and (a second sort)
         PlannerUtil.transformSortTo2PWithStore((SortNode)node, tableId);
       } else if (node.getType() == NodeType.JOIN) {
@@ -138,8 +138,8 @@ public class GlobalPlanner {
         JoinNode join = (JoinNode) node;
 
         /*
-        if (join.getOuterNode().getType() == NodeType.SCAN &&
-            join.getInnerNode().getType() == NodeType.SCAN) {
+        if (join.getOuterNode().getType() == ExprType.SCAN &&
+            join.getInnerNode().getType() == ExprType.SCAN) {
           ScanNode outerScan = (ScanNode) join.getOuterNode();
           ScanNode innerScan = (ScanNode) join.getInnerNode();
 
@@ -198,14 +198,14 @@ public class GlobalPlanner {
         // insert stores for the first phase
         if (join.getLeftChild().getType() != NodeType.UNION &&
             join.getLeftChild().getType() != NodeType.STORE) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           store = new StoreTableNode(tableId);
           store.setLocal(true);
           PlannerUtil.insertOuterNode(node, store);
         }
         if (join.getRightChild().getType() != NodeType.UNION &&
             join.getRightChild().getType() != NodeType.STORE) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           store = new StoreTableNode(tableId);
           store.setLocal(true);
           PlannerUtil.insertInnerNode(node, store);
@@ -216,7 +216,7 @@ public class GlobalPlanner {
         // insert stores
         if (union.getLeftChild().getType() != NodeType.UNION &&
             union.getLeftChild().getType() != NodeType.STORE) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           store = new StoreTableNode(tableId);
           if(union.getLeftChild().getType() == NodeType.GROUP_BY) {
             /*This case is for cube by operator
@@ -230,7 +230,7 @@ public class GlobalPlanner {
         }
         if (union.getRightChild().getType() != NodeType.UNION &&
             union.getRightChild().getType() != NodeType.STORE) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           store = new StoreTableNode(tableId);
           if(union.getRightChild().getType() == NodeType.GROUP_BY) {
             /*This case is for cube by operator
@@ -246,7 +246,7 @@ public class GlobalPlanner {
         UnaryNode unary = (UnaryNode)node;
         if (unary.getType() != NodeType.STORE &&
             unary.getChild().getType() != NodeType.STORE) {
-          tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+          tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
           insertStore(tableId, unary);
         }
       }
@@ -283,11 +283,11 @@ public class GlobalPlanner {
       
       if (node.getType() == NodeType.STORE) {
         store = (StoreTableNode) node;
-        SubQueryId id;
-        if (store.getTableName().startsWith(QueryId.PREFIX)) {
-          id = TajoIdUtils.newSubQueryId(store.getTableName());
+        ExecutionBlockId id;
+        if (store.getTableName().startsWith(ExecutionBlockId.EB_ID_PREFIX)) {
+          id = TajoIdUtils.createExecutionBlockId(store.getTableName());
         } else {
-          id = QueryIdFactory.newSubQueryId(queryId);
+          id = QueryIdFactory.newExecutionBlockId(queryId);
         }
         subQuery = new ExecutionBlock(id);
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
new file mode 100644
index 0000000..8f83557
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.AbstractService;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TajoAsyncDispatcher extends AbstractService  implements Dispatcher {
+
+  private static final Log LOG = LogFactory.getLog(TajoAsyncDispatcher.class);
+
+  private final BlockingQueue<Event> eventQueue;
+  private volatile boolean stopped = false;
+
+  private Thread eventHandlingThread;
+  protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers;
+  private boolean exitOnDispatchException;
+
+  private String id;
+
+  public TajoAsyncDispatcher(String id) {
+    this(id, new LinkedBlockingQueue<Event>());
+  }
+
+  public TajoAsyncDispatcher(String id, BlockingQueue<Event> eventQueue) {
+    super(TajoAsyncDispatcher.class.getName());
+    this.id = id;
+    this.eventQueue = eventQueue;
+    this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>();
+  }
+
+  Runnable createThread() {
+    return new Runnable() {
+      @Override
+      public void run() {
+        while (!stopped && !Thread.currentThread().isInterrupted()) {
+          Event event;
+          try {
+            event = eventQueue.take();
+            if(LOG.isDebugEnabled()) {
+              LOG.debug(id + ",event take:" + event.getType() + "," + event);
+            }
+          } catch(InterruptedException ie) {
+            if (!stopped) {
+              LOG.warn("AsyncDispatcher thread interrupted", ie);
+            }
+            return;
+          }
+          dispatch(event);
+        }
+      }
+    };
+  }
+
+  @Override
+  public synchronized void init(Configuration conf) {
+    this.exitOnDispatchException =
+        conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
+            Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    //start all the components
+    super.start();
+    eventHandlingThread = new Thread(createThread());
+    eventHandlingThread.setName("AsyncDispatcher event handler");
+    eventHandlingThread.start();
+
+    LOG.info("AsyncDispatcher started:" + id);
+  }
+
+  @Override
+  public synchronized void stop() {
+    if(stopped) {
+      return;
+    }
+    stopped = true;
+    if (eventHandlingThread != null) {
+      eventHandlingThread.interrupt();
+      try {
+        eventHandlingThread.join();
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted Exception while stopping", ie);
+      }
+    }
+
+    // stop all the components
+    super.stop();
+
+    LOG.info("AsyncDispatcher stopped:" + id);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected void dispatch(Event event) {
+    //all events go thru this loop
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Dispatching the event " + event.getClass().getName() + "."
+          + event.toString());
+    }
+//    LOG.info("====> Dispatching the event " + event.getClass().getName() + "."
+//        + event.toString() );
+    Class<? extends Enum> type = event.getType().getDeclaringClass();
+
+    try{
+      EventHandler handler = eventDispatchers.get(type);
+      if(handler != null) {
+        handler.handle(event);
+      } else {
+        throw new Exception("No handler for registered for " + type);
+      }
+    } catch (Throwable t) {
+      //TODO Maybe log the state of the queue
+      LOG.fatal("Error in dispatcher thread:" + event.getType(), t);
+      if (exitOnDispatchException && (ShutdownHookManager.get().isShutdownInProgress()) == false) {
+        LOG.info("Exiting, bye..");
+        System.exit(-1);
+      }
+    } finally {
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void register(Class<? extends Enum> eventType,
+                       EventHandler handler) {
+    /* check to see if we have a listener registered */
+    EventHandler<Event> registeredHandler = (EventHandler<Event>)
+        eventDispatchers.get(eventType);
+    LOG.debug("Registering " + eventType + " for " + handler.getClass());
+    if (registeredHandler == null) {
+      eventDispatchers.put(eventType, handler);
+    } else if (!(registeredHandler instanceof MultiListenerHandler)){
+      /* for multiple listeners of an event add the multiple listener handler */
+      MultiListenerHandler multiHandler = new MultiListenerHandler();
+      multiHandler.addHandler(registeredHandler);
+      multiHandler.addHandler(handler);
+      eventDispatchers.put(eventType, multiHandler);
+    } else {
+      /* already a multilistener, just add to it */
+      MultiListenerHandler multiHandler
+          = (MultiListenerHandler) registeredHandler;
+      multiHandler.addHandler(handler);
+    }
+  }
+
+  @Override
+  public EventHandler getEventHandler() {
+    return new GenericEventHandler();
+  }
+
+  class GenericEventHandler implements EventHandler<Event> {
+    public void handle(Event event) {
+      /* all this method does is enqueue all the events onto the queue */
+      int qSize = eventQueue.size();
+      if (qSize !=0 && qSize %1000 == 0) {
+        LOG.info("Size of event-queue is " + qSize);
+      }
+      int remCapacity = eventQueue.remainingCapacity();
+      if (remCapacity < 1000) {
+        LOG.warn("Very low remaining capacity in the event-queue: "
+            + remCapacity);
+      }
+      try {
+        if(LOG.isDebugEnabled()) {
+          LOG.debug(id + ",add event:" +
+              event.getType() + "," + event + "," +
+              (eventHandlingThread == null ? "null" : eventHandlingThread.isAlive()));
+        }
+        eventQueue.put(event);
+      } catch (InterruptedException e) {
+        if (!stopped) {
+          LOG.warn("AsyncDispatcher thread interrupted", e);
+        }
+        throw new YarnException(e);
+      }
+    }
+  }
+
+  /**
+   * Multiplexing an event. Sending it to different handlers that
+   * are interested in the event.
+   */
+  static class MultiListenerHandler implements EventHandler<Event> {
+    List<EventHandler<Event>> listofHandlers;
+
+    public MultiListenerHandler() {
+      listofHandlers = new ArrayList<EventHandler<Event>>();
+    }
+
+    @Override
+    public void handle(Event event) {
+      for (EventHandler<Event> handler: listofHandlers) {
+        handler.handle(event);
+      }
+    }
+
+    void addHandler(EventHandler<Event> handler) {
+      listofHandlers.add(handler);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
new file mode 100644
index 0000000..5359311
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.event.QueryEvent;
+import org.apache.tajo.master.event.QueryEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.rm.TajoWorkerContainer;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.ProtoAsyncRpcClient;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TajoContainerProxy extends ContainerProxy {
+  public TajoContainerProxy(QueryMasterTask.QueryContext context,
+                            Configuration conf, Container container,
+                            ExecutionBlockId executionBlockId) {
+    super(context, conf, executionBlockId, container);
+  }
+
+  @Override
+  public void launch(ContainerLaunchContext containerLaunchContext) {
+    context.getResourceAllocator().addContainer(containerID, this);
+    this.hostName = container.getNodeId().getHost();
+    this.port = context.getQueryMasterContext().getWorkerContext().getPullService().getPort();
+    this.state = ContainerState.RUNNING;
+
+    assignExecutionBlock(executionBlockId, container);
+
+    context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+  }
+
+  private void assignExecutionBlock(ExecutionBlockId executionBlockId, Container container) {
+    ProtoAsyncRpcClient tajoWorkerRpc = null;
+    try {
+      InetSocketAddress myAddr= context.getQueryMasterContext().getWorkerContext()
+          .getTajoWorkerManagerService().getBindAddr();
+
+      InetSocketAddress addr = new InetSocketAddress(container.getNodeId().getHost(), container.getNodeId().getPort());
+      tajoWorkerRpc = new ProtoAsyncRpcClient(TajoWorkerProtocol.class, addr);
+      TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub();
+
+      TajoWorkerProtocol.RunExecutionBlockRequestProto request =
+          TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder()
+              .setExecutionBlockId(executionBlockId.toString())
+              .setQueryMasterHost(myAddr.getHostName())
+              .setQueryMasterPort(myAddr.getPort())
+              .setNodeId(container.getNodeId().toString())
+              .setContainerId(container.getId().toString())
+              .setQueryOutputPath(context.getOutputPath().toString())
+              .build();
+
+      tajoWorkerRpcClient.executeExecutionBlock(null, request, NullCallback.get());
+    } catch (Exception e) {
+      //TODO retry
+      LOG.error(e.getMessage(), e);
+    } finally {
+      if(tajoWorkerRpc != null) {
+        (new AyncRpcClose(tajoWorkerRpc)).start();
+      }
+    }
+  }
+
+  class AyncRpcClose extends Thread {
+    ProtoAsyncRpcClient client;
+    public AyncRpcClose(ProtoAsyncRpcClient client) {
+      this.client = client;
+    }
+
+    @Override
+    public void run() {
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+      }
+      client.close();
+    }
+  }
+
+  @Override
+  public synchronized void stopContainer() {
+    LOG.info("Release TajoWorker Resource: " + executionBlockId + "," + containerID + ", state:" + this.state);
+    if(isCompletelyDone()) {
+      LOG.info("====> Container already stopped:" + containerID);
+      return;
+    }
+    if(this.state == ContainerState.PREP) {
+      this.state = ContainerState.KILLED_BEFORE_LAUNCH;
+    } else {
+      try {
+        releaseWorkerResource(context, executionBlockId, ((TajoWorkerContainer)container).getWorkerResource());
+        context.getResourceAllocator().removeContainer(containerID);
+        this.state = ContainerState.DONE;
+      } catch (Throwable t) {
+        // ignore the cleanup failure
+        String message = "cleanup failed for container "
+            + this.containerID + " : "
+            + StringUtils.stringifyException(t);
+        LOG.warn(message);
+        this.state = ContainerState.DONE;
+        return;
+      }
+    }
+  }
+
+  public static void releaseWorkerResource(QueryMasterTask.QueryContext context,
+                                           ExecutionBlockId executionBlockId,
+                                           WorkerResource workerResource) throws Exception {
+    List<WorkerResource> workerResources = new ArrayList<WorkerResource>();
+    workerResources.add(workerResource);
+
+    releaseWorkerResource(context, executionBlockId, workerResources);
+  }
+
+  public static void releaseWorkerResource(QueryMasterTask.QueryContext context,
+                                           ExecutionBlockId executionBlockId,
+                                           List<WorkerResource> workerResources) throws Exception {
+    List<TajoMasterProtocol.WorkerResourceProto> workerResourceProtos =
+        new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
+
+    for(WorkerResource eahWorkerResource: workerResources) {
+      workerResourceProtos.add(TajoMasterProtocol.WorkerResourceProto.newBuilder()
+          .setWorkerHostAndPort(eahWorkerResource.getId())
+          .setExecutionBlockId(executionBlockId.getProto())
+          .setMemoryMBSlots(eahWorkerResource.getMemoryMBSlots())
+          .setDiskSlots(eahWorkerResource.getDiskSlots())
+          .build()
+      );
+    }
+    context.getQueryMasterContext().getWorkerContext().getTajoMasterRpcClient()
+        .releaseWorkerResource(null,
+            TajoMasterProtocol.WorkerResourceReleaseRequest.newBuilder()
+                .addAllWorkerResources(workerResourceProtos)
+                .build(),
+            NullCallback.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index b84b51b..f22472a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -18,7 +18,6 @@
 
 package org.apache.tajo.master;
 
-import com.google.common.collect.Maps;
 import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,12 +31,9 @@ import org.apache.hadoop.yarn.SystemClock;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.util.RackResolver;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
@@ -47,16 +43,15 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.function.Country;
 import org.apache.tajo.engine.function.InCountry;
 import org.apache.tajo.engine.function.builtin.*;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.master.querymaster.QueryMasterManager;
-import org.apache.tajo.master.querymaster.QueryMasterManagerService;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.master.rm.WorkerResourceManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.webapp.StaticHttpServer;
 
+import java.lang.reflect.Constructor;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 
 public class TajoMaster extends CompositeService {
 
@@ -79,12 +74,14 @@ public class TajoMaster extends CompositeService {
   private GlobalEngine globalEngine;
   private AsyncDispatcher dispatcher;
   private TajoMasterClientService tajoMasterClientService;
-  private QueryMasterManagerService queryMasterManagerService;
-  private YarnRPC yarnRPC;
+  private TajoMasterService tajoMasterService;
 
+  private WorkerResourceManager resourceManager;
   //Web Server
   private StaticHttpServer webServer;
 
+  private QueryJobManager queryJobManager;
+
   public TajoMaster() throws Exception {
     super(TajoMaster.class.getName());
   }
@@ -96,16 +93,23 @@ public class TajoMaster extends CompositeService {
     context = new MasterContext(conf);
     clock = new SystemClock();
 
-
     try {
       RackResolver.init(conf);
 
+//      this.conf.writeXml(System.out);
+      String className = this.conf.get("tajo.resource.manager", TajoWorkerResourceManager.class.getCanonicalName());
+      Class<WorkerResourceManager> resourceManagerClass =
+          (Class<WorkerResourceManager>)Class.forName(className);
+
+      Constructor<WorkerResourceManager> constructor = resourceManagerClass.getConstructor(MasterContext.class);
+      resourceManager = constructor.newInstance(context);
+      resourceManager.init(context.getConf());
+
+      //TODO WebServer port configurable
       webServer = StaticHttpServer.getInstance(this ,"admin", null, 8080 ,
           true, null, context.getConf(), null);
       webServer.start();
 
-      QueryIdFactory.reset();
-
       // Get the tajo base dir
       this.basePath = new Path(conf.getVar(ConfVars.ROOT_DIR));
       LOG.info("Tajo Root dir is set " + basePath);
@@ -128,8 +132,6 @@ public class TajoMaster extends CompositeService {
         LOG.info("Warehouse dir (" + wareHousePath + ") is created");
       }
 
-      yarnRPC = YarnRPC.create(conf);
-
       this.dispatcher = new AsyncDispatcher();
       addIfService(dispatcher);
 
@@ -149,15 +151,19 @@ public class TajoMaster extends CompositeService {
       globalEngine = new GlobalEngine(context);
       addIfService(globalEngine);
 
+      queryJobManager = new QueryJobManager(context);
+      addIfService(queryJobManager);
+
       tajoMasterClientService = new TajoMasterClientService(context);
       addIfService(tajoMasterClientService);
 
-      queryMasterManagerService = new QueryMasterManagerService(context);
-      addIfService(queryMasterManagerService);
+      tajoMasterService = new TajoMasterService(context);
+      addIfService(tajoMasterService);
     } catch (Exception e) {
-       e.printStackTrace();
+       LOG.error(e.getMessage(), e);
     }
 
+    LOG.info("====> Tajo master started");
     super.init(conf);
   }
 
@@ -282,10 +288,6 @@ public class TajoMaster extends CompositeService {
       LOG.error(e);
     }
 
-    for(QueryMasterManager eachQuery: getContext().getAllQueries().values()) {
-      eachQuery.stop();
-    }
-
     super.stop();
     LOG.info("TajoMaster main thread exiting");
   }
@@ -310,40 +312,7 @@ public class TajoMaster extends CompositeService {
     return this.storeManager;
   }
 
-  // TODO - to be improved
-  public Collection<QueryMasterProtocol.TaskStatusProto> getProgressQueries() {
-    return null;
-  }
-
-//  private class QueryEventDispatcher implements EventHandler<QueryEvent> {
-//    @Override
-//    public void handle(QueryEvent queryEvent) {
-//      LOG.info("QueryEvent: " + queryEvent.getQueryId());
-//      LOG.info("Found: " + context.getQuery(queryEvent.getQueryId()).getContext().getQueryId());
-//      context.getQuery(queryEvent.getQueryId()).handle(queryEvent);
-//    }
-//  }
-
-  public static void main(String[] args) throws Exception {
-    StringUtils.startupShutdownMessage(TajoMaster.class, args, LOG);
-
-    try {
-      TajoMaster master = new TajoMaster();
-      ShutdownHookManager.get().addShutdownHook(
-          new CompositeServiceShutdownHook(master),
-          SHUTDOWN_HOOK_PRIORITY);
-      TajoConf conf = new TajoConf(new YarnConfiguration());
-      master.init(conf);
-      master.start();
-    } catch (Throwable t) {
-      LOG.fatal("Error starting TajoMaster", t);
-      System.exit(-1);
-    }
-  }
-
   public class MasterContext {
-    //private final Map<QueryId, QueryMaster> queries = Maps.newConcurrentMap();
-    private final Map<QueryId, QueryMasterManager> queries = Maps.newConcurrentMap();
     private final TajoConf conf;
 
     public MasterContext(TajoConf conf) {
@@ -358,20 +327,12 @@ public class TajoMaster extends CompositeService {
       return clock;
     }
 
-    public QueryMasterManager getQuery(QueryId queryId) {
-      return queries.get(queryId);
+    public QueryJobManager getQueryJobManager() {
+      return queryJobManager;
     }
 
-    public Map<QueryId, QueryMasterManager> getAllQueries() {
-      return queries;
-    }
-
-    public void addQuery(QueryId queryId, QueryMasterManager queryMasterManager) {
-      queries.put(queryId, queryMasterManager);
-    }
-
-    public AsyncDispatcher getDispatcher() {
-      return dispatcher;
+    public WorkerResourceManager getResourceManager() {
+      return resourceManager;
     }
 
     public EventHandler getEventHandler() {
@@ -390,16 +351,23 @@ public class TajoMaster extends CompositeService {
       return storeManager;
     }
 
-    public YarnRPC getYarnRPC() {
-      return yarnRPC;
+    public TajoMasterService getTajoMasterService() {
+      return tajoMasterService;
     }
+  }
 
-    public TajoMasterClientService getClientService() {
-      return tajoMasterClientService;
-    }
+  public static void main(String[] args) throws Exception {
+    StringUtils.startupShutdownMessage(TajoMaster.class, args, LOG);
 
-    public QueryMasterManagerService getQueryMasterManagerService() {
-      return queryMasterManagerService;
+    try {
+      TajoMaster master = new TajoMaster();
+      ShutdownHookManager.get().addShutdownHook(new CompositeServiceShutdownHook(master), SHUTDOWN_HOOK_PRIORITY);
+      TajoConf conf = new TajoConf(new YarnConfiguration());
+      master.init(conf);
+      master.start();
+    } catch (Throwable t) {
+      LOG.fatal("Error starting TajoMaster", t);
+      System.exit(-1);
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index ed1376c..8578b64 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -26,9 +26,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoIdProtos;
 import org.apache.tajo.TajoProtos;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.exception.AlreadyExistsTableException;
@@ -37,18 +39,18 @@ import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
 import org.apache.tajo.catalog.statistics.TableStat;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.query.exception.SQLSyntaxError;
+import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.ipc.TajoMasterClientProtocol;
 import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService;
 import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.querymaster.QueryMasterManager;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.querymaster.QueryJobManager;
 import org.apache.tajo.rpc.ProtoBlockingRpcServer;
 import org.apache.tajo.rpc.RemoteException;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -88,9 +90,10 @@ public class TajoMasterClientService extends AbstractService {
       LOG.error(e);
     }
     server.start();
-    bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
-    this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress));
-    LOG.info("TajoMasterClientService startup");
+    bindAddress = server.getListenAddress();
+    this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS,
+        org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
+    LOG.info("Instantiated TajoMasterClientService at " + this.bindAddress);
     super.start();
   }
 
@@ -99,7 +102,6 @@ public class TajoMasterClientService extends AbstractService {
     if (server != null) {
       server.shutdown();
     }
-    LOG.info("TajoMasterClientService shutdown");
     super.stop();
   }
 
@@ -123,40 +125,26 @@ public class TajoMasterClientService extends AbstractService {
     }
 
     @Override
-    public SubmitQueryResponse submitQuery(RpcController controller,
+    public GetQueryStatusResponse submitQuery(RpcController controller,
                                            QueryRequest request)
         throws ServiceException {
 
-      QueryId queryId;
-      SubmitQueryResponse.Builder build = SubmitQueryResponse.newBuilder();
       try {
-        queryId = context.getGlobalEngine().executeQuery(request.getQuery());
-      } catch (SQLSyntaxError e) {
-        build.setResultCode(ResultCode.ERROR);
-        build.setErrorMessage(e.getMessage());
-        return build.build();
-
-      } catch (Exception e) {
-        build.setResultCode(ResultCode.ERROR);
-        String msg = e.getMessage();
-        if (msg == null) {
-          msg = "Internal Error";
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Query [" + request.getQuery() + "] is submitted");
         }
-
-        if (LOG.isDebugEnabled()) {
-          LOG.error(msg, e);
+        return context.getGlobalEngine().executeQuery(request.getQuery());
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        ClientProtos.GetQueryStatusResponse.Builder responseBuilder = ClientProtos.GetQueryStatusResponse.newBuilder();
+        responseBuilder.setResultCode(ResultCode.ERROR);
+        if (e.getMessage() != null) {
+          responseBuilder.setErrorMessage(ExceptionUtils.getStackTrace(e));
         } else {
-          LOG.error(msg);
+          responseBuilder.setErrorMessage("Internal Error");
         }
-        build.setErrorMessage(msg);
-        return build.build();
+        return responseBuilder.build();
       }
-
-      LOG.info("Query " + queryId + " is submitted");
-      build.setResultCode(ResultCode.OK);
-      build.setQueryId(queryId.getProto());
-
-      return build.build();
     }
 
     @Override
@@ -183,13 +171,17 @@ public class TajoMasterClientService extends AbstractService {
                                                  GetQueryResultRequest request)
         throws ServiceException {
       QueryId queryId = new QueryId(request.getQueryId());
-      QueryMasterManager queryMasterManager = context.getQuery(queryId);
+      if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
 
+      }
+      QueryInProgress queryInProgress = context.getQueryJobManager().getQueryInProgress(queryId);
+      QueryInfo queryInfo = queryInProgress.getQueryInfo();
       GetQueryResultResponse.Builder builder
           = GetQueryResultResponse.newBuilder();
-      switch (queryMasterManager.getState()) {
+      switch (queryInfo.getQueryState()) {
         case QUERY_SUCCEEDED:
-          builder.setTableDesc((TableDescProto) queryMasterManager.getResultDesc().getProto());
+          // TODO check this logic needed
+          //builder.setTableDesc((TableDescProto) queryJobManager.getResultDesc().getProto());
           break;
         case QUERY_FAILED:
         case QUERY_ERROR:
@@ -218,23 +210,25 @@ public class TajoMasterClientService extends AbstractService {
       QueryId queryId = new QueryId(request.getQueryId());
       builder.setQueryId(request.getQueryId());
 
-      if (queryId.equals(TajoIdUtils.NullQueryId)) {
+      if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
         builder.setResultCode(ResultCode.OK);
         builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
       } else {
-        QueryMasterManager queryMasterManager = context.getQuery(queryId);
-        if (queryMasterManager != null) {
+        QueryInProgress queryInProgress = context.getQueryJobManager().getQueryInProgress(queryId);
+        if (queryInProgress != null) {
+          QueryInfo queryInfo = queryInProgress.getQueryInfo();
           builder.setResultCode(ResultCode.OK);
-          builder.setState(queryMasterManager.getState());
-          builder.setProgress(queryMasterManager.getProgress());
-          builder.setSubmitTime(queryMasterManager.getAppSubmitTime());
-          if(queryMasterManager.getQueryMasterHost() != null) {
-            builder.setQueryMasterHost(queryMasterManager.getQueryMasterHost());
-            builder.setQueryMasterPort(queryMasterManager.getQueryMasterClientPort());
+          builder.setState(queryInfo.getQueryState());
+          builder.setProgress(queryInfo.getProgress());
+          builder.setSubmitTime(queryInfo.getStartTime());
+          if(queryInfo.getQueryMasterHost() != null) {
+            builder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+            builder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
           }
-
-          if (queryMasterManager.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
-            builder.setFinishTime(queryMasterManager.getFinishTime());
+          //builder.setInitTime(queryJobManager.getInitializationTime());
+          //builder.setHasResult(!queryJobManager.isCreateTableStmt());
+          if (queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+            builder.setFinishTime(queryInfo.getFinishTime());
           } else {
             builder.setFinishTime(System.currentTimeMillis());
           }
@@ -249,11 +243,12 @@ public class TajoMasterClientService extends AbstractService {
 
     @Override
     public BoolProto killQuery(RpcController controller,
-                               ApplicationAttemptIdProto request)
+                               TajoIdProtos.QueryIdProto request)
         throws ServiceException {
       QueryId queryId = new QueryId(request);
-      QueryMasterManager queryMasterManager = context.getQuery(queryId);
-      //queryMasterManager.handle(new QueryEvent(queryId, QueryEventType.KILL));
+      QueryJobManager queryJobManager = context.getQueryJobManager();
+      //TODO KHJ, change QueryJobManager to event handler
+      //queryJobManager.handle(new QueryEvent(queryId, QueryEventType.KILL));
 
       return BOOL_TRUE;
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
new file mode 100644
index 0000000..f0a4618
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.ProtoAsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+
+public class TajoMasterService extends AbstractService {
+  private final static Log LOG = LogFactory.getLog(TajoMasterService.class);
+
+  private final TajoMaster.MasterContext context;
+  private final TajoConf conf;
+  private final TajoMasterServiceHandler masterHandler;
+  private ProtoAsyncRpcServer server;
+  private InetSocketAddress bindAddress;
+
+  private final BoolProto BOOL_TRUE = BoolProto.newBuilder().setValue(true).build();
+  private final BoolProto BOOL_FALSE = BoolProto.newBuilder().setValue(false).build();
+
+  public TajoMasterService(TajoMaster.MasterContext context) {
+    super(TajoMasterService.class.getName());
+    this.context = context;
+    this.conf = context.getConf();
+    this.masterHandler = new TajoMasterServiceHandler();
+  }
+
+  @Override
+  public void start() {
+    // TODO resolve hostname
+    String confMasterServiceAddr = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_SERVICE_ADDRESS);
+    InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr);
+    try {
+      server = new ProtoAsyncRpcServer(TajoMasterProtocol.class, masterHandler, initIsa);
+    } catch (Exception e) {
+      LOG.error(e);
+    }
+    server.start();
+    bindAddress = server.getListenAddress();
+    this.conf.setVar(TajoConf.ConfVars.TAJO_MASTER_SERVICE_ADDRESS,
+        org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
+    LOG.info("Instantiated TajoMasterService at " + this.bindAddress);
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(server != null) {
+      server.shutdown();
+      server = null;
+    }
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddress() {
+    return bindAddress;
+  }
+
+  public class TajoMasterServiceHandler
+      implements TajoMasterProtocol.TajoMasterProtocolService.Interface {
+    @Override
+    public void heartbeat(
+        RpcController controller,
+        TajoMasterProtocol.TajoHeartbeat request, RpcCallback<TajoMasterProtocol.TajoHeartbeatResponse> done) {
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Received QueryHeartbeat:" + request.getTajoWorkerHost() + ":" + request.getTajoWorkerPort());
+      }
+
+      TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand command = null;
+      if(request.hasQueryId()) {
+        QueryId queryId = new QueryId(request.getQueryId());
+
+        //heartbeat from querymaster
+        //LOG.info("Received QueryHeartbeat:" + queryId + "," + request);
+        QueryJobManager queryJobManager = context.getQueryJobManager();
+        command = queryJobManager.queryHeartbeat(request);
+      } else {
+        //heartbeat from TajoWorker
+        context.getResourceManager().workerHeartbeat(request);
+      }
+
+      //ApplicationAttemptId attemptId = queryJobManager.getAppAttemptId();
+      //String attemptIdStr = attemptId == null ? null : attemptId.toString();
+      TajoMasterProtocol.TajoHeartbeatResponse.Builder builder = TajoMasterProtocol.TajoHeartbeatResponse.newBuilder();
+      builder.setHeartbeatResult(BOOL_TRUE);
+      if(command != null) {
+        builder.setResponseCommand(command);
+      }
+      done.run(builder.build());
+    }
+
+    @Override
+    public void allocateWorkerResources(
+        RpcController controller,
+        TajoMasterProtocol.WorkerResourceAllocationRequest request,
+        RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> done) {
+      context.getResourceManager().allocateWorkerResources(request, done);
+
+//      List<String> workerHosts = new ArrayList<String>();
+//      for(WorkerResource eachWorker: workerResources) {
+//        workerHosts.add(eachWorker.getAllocatedHost() + ":" + eachWorker.getPorts()[0]);
+//      }
+//
+//      done.run(TajoMasterProtocol.WorkerResourceAllocationResponse.newBuilder()
+//          .setExecutionBlockId(request.getExecutionBlockId())
+//          .addAllAllocatedWorks(workerHosts)
+//          .build()
+//      );
+    }
+
+    @Override
+    public void releaseWorkerResource(RpcController controller,
+                                           TajoMasterProtocol.WorkerResourceReleaseRequest request,
+                                           RpcCallback<PrimitiveProtos.BoolProto> done) {
+      List<TajoMasterProtocol.WorkerResourceProto> workerResources = request.getWorkerResourcesList();
+      for(TajoMasterProtocol.WorkerResourceProto eachWorkerResource: workerResources) {
+        WorkerResource workerResource = new WorkerResource();
+        String[] tokens = eachWorkerResource.getWorkerHostAndPort().split(":");
+        workerResource.setAllocatedHost(tokens[0]);
+        workerResource.setPorts(new int[]{Integer.parseInt(tokens[1])});
+        workerResource.setMemoryMBSlots(eachWorkerResource.getMemoryMBSlots());
+        workerResource.setDiskSlots(eachWorkerResource.getDiskSlots());
+
+        LOG.info("====> releaseWorkerResource:" + workerResource);
+        context.getResourceManager().releaseWorkerResource(
+            new QueryId(eachWorkerResource.getExecutionBlockId().getQueryId()),
+            workerResource);
+      }
+      done.run(BOOL_TRUE);
+    }
+
+    @Override
+    public void stopQueryMaster(RpcController controller, TajoIdProtos.QueryIdProto request,
+                                RpcCallback<BoolProto> done) {
+      context.getQueryJobManager().stopQuery(new QueryId(request));
+      done.run(BOOL_TRUE);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
index 93aaa5d..1e6655c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
@@ -20,7 +20,7 @@ package org.apache.tajo.master;
 
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
 
 import java.util.Collection;
@@ -31,17 +31,21 @@ public class TaskRunnerGroupEvent extends AbstractEvent<EventType> {
     CONTAINER_REMOTE_CLEANUP
   }
 
-  protected final SubQueryId subQueryId;
+  protected final ExecutionBlockId executionBlockId;
   protected final Collection<Container> containers;
   public TaskRunnerGroupEvent(EventType eventType,
-                              SubQueryId subQueryId,
+                              ExecutionBlockId executionBlockId,
                               Collection<Container> containers) {
     super(eventType);
-    this.subQueryId = subQueryId;
+    this.executionBlockId = executionBlockId;
     this.containers = containers;
   }
 
   public Collection<Container> getContainers() {
     return containers;
   }
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
 }


[2/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
index a41b280..47ec7bc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
@@ -26,20 +26,20 @@ import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryConf;
+import org.apache.tajo.QueryId;
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.TajoProtos.TaskAttemptState;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.query.QueryUnitRequestImpl;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService.Interface;
 import org.apache.tajo.rpc.CallFuture2;
 import org.apache.tajo.rpc.NullCallback;
 import org.apache.tajo.rpc.ProtoAsyncRpcClient;
@@ -51,7 +51,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Map;
 import java.util.concurrent.*;
 
-import static org.apache.tajo.ipc.QueryMasterProtocol.*;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
 
 /**
  * The driver class for Tajo QueryUnit processing.
@@ -60,17 +60,17 @@ public class TaskRunner extends AbstractService {
   /** class logger */
   private static final Log LOG = LogFactory.getLog(TaskRunner.class);
 
-  private QueryConf conf;
+  private QueryConf queryConf;
 
   private volatile boolean stopped = false;
 
-  private final SubQueryId subQueryId;
-  private ApplicationId appId;
-  private final NodeId nodeId;
-  private final ContainerId containerId;
+  private ExecutionBlockId executionBlockId;
+  private QueryId queryId;
+  private NodeId nodeId;
+  private ContainerId containerId;
 
   // Cluster Management
-  private QueryMasterProtocol.QueryMasterProtocolService.Interface master;
+  private TajoWorkerProtocol.TajoWorkerProtocolService.Interface master;
 
   // for temporal or intermediate files
   private FileSystem localFS;
@@ -94,7 +94,7 @@ public class TaskRunner extends AbstractService {
   private Thread taskLauncher;
 
   // Contains the object references related for TaskRunner
-  private WorkerContext workerContext;
+  private TaskRunnerContext taskRunnerContext;
   // for the doAs block
   private UserGroupInformation taskOwner;
 
@@ -102,34 +102,89 @@ public class TaskRunner extends AbstractService {
   private String baseDir;
   private Path baseDirPath;
 
+  private ProtoAsyncRpcClient client;
+
+  private TaskRunnerManager taskRunnerManager;
+
   public TaskRunner(
-      final SubQueryId subQueryId,
+      final ExecutionBlockId executionBlockId,
       final NodeId nodeId,
       UserGroupInformation taskOwner,
       Interface master, ContainerId containerId) {
     super(TaskRunner.class.getName());
-    this.subQueryId = subQueryId;
-    this.appId = subQueryId.getQueryId().getApplicationId();
+    this.executionBlockId = executionBlockId;
+    this.queryId = executionBlockId.getQueryId();
     this.nodeId = nodeId;
     this.taskOwner = taskOwner;
     this.master = master;
     this.containerId = containerId;
   }
 
-  @Override
-  public void init(Configuration _conf) {
-    this.conf = (QueryConf) _conf;
+  public TaskRunner(TaskRunnerManager taskRunnerManager, QueryConf conf, String[] args) {
+    super(TaskRunner.class.getName());
 
+    this.taskRunnerManager = taskRunnerManager;
     try {
-      this.workerContext = new WorkerContext();
+      final ExecutionBlockId executionBlockId = TajoIdUtils.createExecutionBlockId(args[1]);
+
+      conf.setOutputPath(new Path(args[6]));
+
+      LOG.info("NM Local Dir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname));
+      LOG.info("OUTPUT DIR: " + conf.getOutputPath());
+      LOG.info("Tajo Root Dir: " + conf.getVar(ConfVars.ROOT_DIR));
+
+      UserGroupInformation.setConfiguration(conf);
+
+      // QueryBlockId from String
+      // NodeId has a form of hostname:port.
+      NodeId nodeId = ConverterUtils.toNodeId(args[2]);
+      this.containerId = ConverterUtils.toContainerId(args[3]);
+
+      // QueryMaster's address
+      String host = args[4];
+      int port = Integer.parseInt(args[5]);
+      final InetSocketAddress masterAddr = NetUtils.createSocketAddrForHost(host, port);
+
+      LOG.info("QueryMaster Address:" + masterAddr);
+      // TODO - 'load credential' should be implemented
+      // Getting taskOwner
+      UserGroupInformation taskOwner =
+          UserGroupInformation.createRemoteUser(conf.getVar(ConfVars.QUERY_USERNAME));
+      //taskOwner.addToken(token);
+
+      // initialize MasterWorkerProtocol as an actual task owner.
+      this.client =
+          taskOwner.doAs(new PrivilegedExceptionAction<ProtoAsyncRpcClient>() {
+            @Override
+            public ProtoAsyncRpcClient run() throws Exception {
+              return new ProtoAsyncRpcClient(TajoWorkerProtocol.class, masterAddr);
+            }
+          });
+      this.master = client.getStub();
+
+      this.executionBlockId = executionBlockId;
+      this.queryId = executionBlockId.getQueryId();
+      this.nodeId = nodeId;
+      this.taskOwner = taskOwner;
+
+      this.taskRunnerContext = new TaskRunnerContext();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.queryConf = (QueryConf)conf;
 
+    try {
       // initialize DFS and LocalFileSystems
-      defaultFS = FileSystem.get(URI.create(conf.getVar(ConfVars.ROOT_DIR)),conf);
+      defaultFS = FileSystem.get(URI.create(queryConf.getVar(ConfVars.ROOT_DIR)),conf);
       localFS = FileSystem.getLocal(conf);
 
       // the base dir for an output dir
-      baseDir = ConverterUtils.toString(appId)
-          + "/output" + "/" + subQueryId.getId();
+      baseDir = queryId.toString()
+          + "/output" + "/" + executionBlockId.getId();
 
       // initialize LocalDirAllocator
       lDirAllocator = new LocalDirAllocator(ConfVars.TASK_LOCAL_DIR.varname);
@@ -139,9 +194,7 @@ public class TaskRunner extends AbstractService {
 
       // Setup QueryEngine according to the query plan
       // Here, we can setup row-based query engine or columnar query engine.
-      this.queryEngine = new TajoQueryEngine(conf);
-
-      Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+      this.queryEngine = new TajoQueryEngine(queryConf);
     } catch (Throwable t) {
       LOG.error(t);
     }
@@ -152,39 +205,46 @@ public class TaskRunner extends AbstractService {
   @Override
   public void start() {
     run();
+    super.start();
   }
 
   @Override
   public void stop() {
-    if (!isStopped()) {
-      // If TaskRunner is stopped, all running or pending tasks will be marked as failed.
-      for (Task task : tasks.values()) {
-        if (task.getStatus() == TaskAttemptState.TA_PENDING ||
-            task.getStatus() == TaskAttemptState.TA_RUNNING) {
-          task.setState(TaskAttemptState.TA_FAILED);
-        }
+    if(isStopped()) {
+      return;
+    }
+    // If this flag become true, taskLauncher will be terminated.
+    this.stopped = true;
+
+    // If TaskRunner is stopped, all running or pending tasks will be marked as failed.
+    for (Task task : tasks.values()) {
+      if (task.getStatus() == TaskAttemptState.TA_PENDING ||
+          task.getStatus() == TaskAttemptState.TA_RUNNING) {
+        task.setState(TaskAttemptState.TA_FAILED);
       }
+    }
 
-      // If this flag become true, taskLauncher will be terminated.
-      this.stopped = true;
+    if(client != null) {
+      client.close();
+      client = null;
+    }
 
-      LOG.info("STOPPED: " + nodeId);
-      synchronized (this) {
-        notifyAll();
-      }
+    LOG.info("Stop TaskRunner: " + executionBlockId);
+    synchronized (this) {
+      notifyAll();
     }
   }
 
-  class WorkerContext {
-    public QueryConf getConf() {
-      return conf;
+  public class TaskRunnerContext {
+    public QueryConf getQueryConf() {
+      return queryConf;
     }
 
     public String getNodeId() {
       return nodeId.toString();
     }
 
-    public QueryMasterProtocolService.Interface getMaster() {
+    public TajoWorkerProtocolService.Interface getMaster() {
       return master;
     }
 
@@ -219,9 +279,17 @@ public class TaskRunner extends AbstractService {
     public Path getBaseDir() {
       return baseDirPath;
     }
+
+    public ExecutionBlockId getExecutionBlockId() {
+      return executionBlockId;
+    }
+  }
+
+  public TaskRunnerContext getContext() {
+    return taskRunnerContext;
   }
 
-  static void fatalError(QueryMasterProtocolService.Interface proxy,
+  static void fatalError(TajoWorkerProtocolService.Interface proxy,
                          QueryUnitAttemptId taskAttemptId, String message) {
     TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder()
         .setId(taskAttemptId.getProto())
@@ -245,17 +313,27 @@ public class TaskRunner extends AbstractService {
             try {
               if (callFuture == null) {
                 callFuture = new CallFuture2<QueryUnitRequestProto>();
-                master.getTask(null, ((ContainerIdPBImpl) containerId).getProto(),
-                    callFuture);
+                LOG.info("====>Request GetTask:" + executionBlockId + "," + containerId);
+                GetTaskRequestProto request = GetTaskRequestProto.newBuilder()
+                    .setExecutionBlockId(executionBlockId.getProto())
+                    .setContainerId(((ContainerIdPBImpl) containerId).getProto())
+                    .build();
+                master.getTask(null, request, callFuture);
               }
               try {
                 // wait for an assigning task for 3 seconds
                 taskRequest = callFuture.get(3, TimeUnit.SECONDS);
+              } catch (InterruptedException e) {
+                if(stopped) {
+                  break;
+                }
               } catch (TimeoutException te) {
+                if(stopped) {
+                  break;
+                }
                 // if there has been no assigning task for a given period,
                 // TaskRunner will retry to request an assigning task.
-                LOG.error(te);
-
+                LOG.warn("Timeout getResource:" + executionBlockId + ", but retry", te);
                 continue;
               }
 
@@ -264,9 +342,12 @@ public class TaskRunner extends AbstractService {
                 // If TaskRunner receives the terminal signal, TaskRunner will be terminated
                 // immediately.
                 if (taskRequest.getShouldDie()) {
-                  LOG.info("received ShouldDie flag");
+                  LOG.info("Received ShouldDie flag:" + executionBlockId);
                   stop();
-
+                  if(taskRunnerManager != null) {
+                    //notify to TaskRunnerManager
+                    taskRunnerManager.stopTask(executionBlockId);
+                  }
                 } else {
 
                   LOG.info("Accumulated Received Task: " + (++receivedNum));
@@ -280,7 +361,7 @@ public class TaskRunner extends AbstractService {
                   LOG.info("Initializing: " + taskAttemptId);
                   Task task;
                   try {
-                    task = new Task(taskAttemptId, workerContext, master,
+                    task = new Task(taskAttemptId, taskRunnerContext, master,
                         new QueryUnitRequestImpl(taskRequest));
                     tasks.put(taskAttemptId, task);
 
@@ -291,7 +372,7 @@ public class TaskRunner extends AbstractService {
                     // task.run() is a blocking call.
                     task.run();
                   } catch (Throwable t) {
-                    fatalError(workerContext.getMaster(), taskAttemptId, t.getMessage());
+                    fatalError(taskRunnerContext.getMaster(), taskAttemptId, t.getMessage());
                   } finally {
                     callFuture = null;
                     taskRequest = null;
@@ -318,14 +399,6 @@ public class TaskRunner extends AbstractService {
     }
   }
 
-  private class ShutdownHook implements Runnable {
-    @Override
-    public void run() {
-      LOG.info("received SIGINT Signal");
-      stop();
-    }
-  }
-
   /**
    * @return true if a stop has been requested.
    */
@@ -333,68 +406,7 @@ public class TaskRunner extends AbstractService {
     return this.stopped;
   }
 
-  /**
-   * TaskRunner takes 5 arguments as follows:
-   * <ol>
-   * <li>1st: SubQueryId</li>
-   * <li>2nd: NodeId</li>
-   * <li>3nd: ContainerId</li>
-   * <li>4th: QueryMaster hostname</li>
-   * <li>5th: QueryMaster port</li>
-   * </ol>
-   */
-  public static void main(String[] args) throws Exception {
-    // Restore QueryConf
-    final QueryConf conf = new QueryConf();
-    conf.addResource(new Path(QueryConf.FILENAME));
-
-    LOG.info("MiniTajoYarn NM Local Dir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname));
-    LOG.info("OUTPUT DIR: " + conf.getOutputPath());
-    LOG.info("Tajo Root Dir: " + conf.getVar(ConfVars.ROOT_DIR));
-
-    UserGroupInformation.setConfiguration(conf);
-
-    // SubQueryId from String
-    final SubQueryId subQueryId = TajoIdUtils.newSubQueryId(args[0]);
-    // NodeId has a form of hostname:port.
-    NodeId nodeId = ConverterUtils.toNodeId(args[1]);
-    ContainerId containerId = ConverterUtils.toContainerId(args[2]);
-
-    // QueryMaster's address
-    String host = args[3];
-    int port = Integer.parseInt(args[4]);
-    final InetSocketAddress masterAddr =
-            NetUtils.createSocketAddrForHost(host, port);
-
-    // TODO - 'load credential' should be implemented
-    // Getting taskOwner
-    UserGroupInformation taskOwner =
-        UserGroupInformation.createRemoteUser(conf.getVar(ConfVars.QUERY_USERNAME));
-    //taskOwner.addToken(token);
-
-    // QueryMasterService RPC
-    ProtoAsyncRpcClient client;
-    QueryMasterProtocolService.Interface master;
-
-    // initialize MasterWorkerProtocol as an actual task owner.
-    client =
-        taskOwner.doAs(new PrivilegedExceptionAction<ProtoAsyncRpcClient>() {
-          @Override
-          public ProtoAsyncRpcClient run() throws Exception {
-            return new ProtoAsyncRpcClient(QueryMasterProtocol.class, masterAddr);
-          }
-        });
-    master = client.getStub();
-
-
-    TaskRunner taskRunner = new TaskRunner(subQueryId, nodeId, taskOwner, master, containerId);
-    try {
-      taskRunner.init(conf);
-      taskRunner.start();
-    } finally {
-      client.close();
-      LOG.info("TaskRunner (" + nodeId + ") main thread exiting");
-      System.exit(0);
-    }
+  public ExecutionBlockId getExecutionBlockId() {
+    return this.executionBlockId;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
new file mode 100644
index 0000000..dcd44df
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.conf.TajoConf;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TaskRunnerManager extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class);
+
+  private Map<ExecutionBlockId, TaskRunner> taskRunnerMap = new HashMap<ExecutionBlockId, TaskRunner>();
+  private TajoWorker.WorkerContext workerContext;
+  private TajoConf tajoConf;
+  private AtomicBoolean stop = new AtomicBoolean(false);
+
+  public TaskRunnerManager(TajoWorker.WorkerContext workerContext) {
+    super(TaskRunnerManager.class.getName());
+
+    this.workerContext = workerContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    tajoConf = (TajoConf)conf;
+    super.init(tajoConf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(stop.get()) {
+      return;
+    }
+    stop.set(true);
+    synchronized(taskRunnerMap) {
+      for(TaskRunner eachTaskRunner: taskRunnerMap.values()) {
+        if(!eachTaskRunner.isStopped()) {
+          eachTaskRunner.stop();
+        }
+      }
+    }
+    super.stop();
+    if(!workerContext.isStandbyMode()) {
+      workerContext.stopWorker(true);
+    }
+  }
+
+  public void stopTask(ExecutionBlockId executionBlockId) {
+    LOG.info("Stop Task:" + executionBlockId);
+    synchronized(taskRunnerMap) {
+      taskRunnerMap.remove(executionBlockId);
+    }
+    if(!workerContext.isStandbyMode()) {
+      stop();
+    }
+  }
+
+  public void startTask(final String[] params) {
+    //TODO change to use event dispatcher
+    Thread t = new Thread() {
+      public void run() {
+        try {
+          QueryConf queryConf = new QueryConf(tajoConf);
+          TaskRunner taskRunner = new TaskRunner(TaskRunnerManager.this, queryConf, params);
+          synchronized(taskRunnerMap) {
+            taskRunnerMap.put(taskRunner.getContext().getExecutionBlockId(), taskRunner);
+          }
+          taskRunner.init(queryConf);
+          taskRunner.start();
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+          throw new RuntimeException(e.getMessage(), e);
+        }
+      }
+    };
+
+    t.start();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
new file mode 100644
index 0000000..9470a88
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.client.YarnClient;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+import org.apache.tajo.master.TaskRunnerLauncher;
+import org.apache.tajo.master.YarnTaskRunnerLauncherImpl;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.rm.YarnRMContainerAllocator;
+
+public class YarnResourceAllocator extends AbstractResourceAllocator {
+  private YarnRMContainerAllocator rmAllocator;
+
+  private TaskRunnerLauncher taskRunnerLauncher;
+
+  private YarnRPC yarnRPC;
+
+  private YarnClient yarnClient;
+
+  private static final Log LOG = LogFactory.getLog(YarnResourceAllocator.class.getName());
+
+  private QueryMasterTask.QueryContext queryContext;
+
+  private QueryConf queryConf;
+
+  public YarnResourceAllocator(QueryMasterTask.QueryContext queryContext) {
+    this.queryContext = queryContext;
+  }
+
+  @Override
+  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId) {
+    return new ContainerIdPBImpl(containerId);
+  }
+
+  @Override
+  public void allocateTaskWorker() {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    queryConf = (QueryConf)conf;
+
+    yarnRPC = YarnRPC.create(queryConf);
+
+    connectYarnClient();
+
+    taskRunnerLauncher = new YarnTaskRunnerLauncherImpl(queryContext, yarnRPC);
+    addService((org.apache.hadoop.yarn.service.Service) taskRunnerLauncher);
+    queryContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, taskRunnerLauncher);
+
+    rmAllocator = new YarnRMContainerAllocator(queryContext);
+    addService(rmAllocator);
+    queryContext.getDispatcher().register(ContainerAllocatorEventType.class, rmAllocator);
+    super.init(conf);
+  }
+
+  @Override
+  public void stop() {
+    try {
+      this.yarnClient.stop();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  private void connectYarnClient() {
+    this.yarnClient = new YarnClientImpl();
+    this.yarnClient.init(queryConf);
+    this.yarnClient.start();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
index d602d57..2ef0c4c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
@@ -22,14 +22,14 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.util.TajoIdUtils;
 import org.apache.tajo.worker.dataserver.FileAccessForbiddenException;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -81,7 +81,7 @@ public class AdvancedDataRetriever implements DataRetriever {
       List<String> qids = splitMaps(params.get("qid"));
       for (String qid : qids) {
         String[] ids = qid.split("_");
-        SubQueryId suid = TajoIdUtils.newSubQueryId(params.get("sid").get(0));
+        ExecutionBlockId suid = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
         QueryUnitId quid = new QueryUnitId(suid, Integer.parseInt(ids[0]));
         QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid,
             Integer.parseInt(ids[1]));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
index 61c14c4..43d99ef 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
@@ -51,13 +51,13 @@ message UpdateQueryResponse {
 
 message SubmitQueryResponse {
   required ResultCode resultCode = 1;
-  optional ApplicationAttemptIdProto queryId = 2;
+  optional string queryId = 2;
   optional string errorMessage = 3;
 }
 
 message GetQueryResultRequest {
   optional SessionIdProto sessionId = 1;
-  required ApplicationAttemptIdProto queryId = 2;
+  required string queryId = 2;
 }
 
 message GetQueryResultResponse {
@@ -70,7 +70,7 @@ message GetQueryListRequest {
 }
 
 message BriefQueryStatus {
-  required ApplicationAttemptIdProto queryId = 1;
+  required string queryId = 1;
   required QueryState state = 2;
   required int32 executionTime = 3;
 }
@@ -81,12 +81,12 @@ message GetQueryListResponse {
 
 message GetQueryStatusRequest {
   optional SessionIdProto sessionId = 1;
-  required ApplicationAttemptIdProto queryId = 2;
+  required string queryId = 2;
 }
 
 message GetQueryStatusResponse {
   required ResultCode resultCode = 1;
-  required ApplicationAttemptIdProto queryId = 2;
+  required string queryId = 2;
   optional QueryState state = 3;
   optional float progress = 4;
   optional int64 submitTime = 5;
@@ -142,7 +142,7 @@ service ClientProtocolService {
   rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
   rpc getQueryList(GetQueryListRequest) returns (GetQueryListResponse);
   rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
-  rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+  rpc killQuery(StringProto) returns (BoolProto);
   rpc getClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse);
   rpc existTable(StringProto) returns (BoolProto);
   rpc getTableList(GetTableListRequest) returns (GetTableListResponse);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
index 2c5c2b6..f3b1005 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
@@ -51,13 +51,13 @@ message UpdateQueryResponse {
 
 message SubmitQueryResponse {
   required ResultCode resultCode = 1;
-  optional ApplicationAttemptIdProto queryId = 2;
+  optional QueryIdProto queryId = 2;
   optional string errorMessage = 3;
 }
 
 message GetQueryResultRequest {
   optional SessionIdProto sessionId = 1;
-  required ApplicationAttemptIdProto queryId = 2;
+  required QueryIdProto queryId = 2;
 }
 
 message GetQueryResultResponse {
@@ -70,7 +70,7 @@ message GetQueryListRequest {
 }
 
 message BriefQueryStatus {
-  required ApplicationAttemptIdProto queryId = 1;
+  required QueryIdProto queryId = 1;
   required QueryState state = 2;
   required int32 executionTime = 3;
 }
@@ -81,12 +81,12 @@ message GetQueryListResponse {
 
 message GetQueryStatusRequest {
   optional SessionIdProto sessionId = 1;
-  required ApplicationAttemptIdProto queryId = 2;
+  required QueryIdProto queryId = 2;
 }
 
 message GetQueryStatusResponse {
   required ResultCode resultCode = 1;
-  required ApplicationAttemptIdProto queryId = 2;
+  required QueryIdProto queryId = 2;
   optional QueryState state = 3;
   optional float progress = 4;
   optional int64 submitTime = 5;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
index 9337078..7da83bc 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
@@ -32,5 +32,5 @@ service QueryMasterClientProtocolService {
   rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
   rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
   rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
-  rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+  rpc killQuery(QueryIdProto) returns (BoolProto);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
deleted file mode 100644
index 08fc5c9..0000000
--- a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tajo.ipc";
-option java_outer_classname = "QueryMasterManagerProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "yarn_protos.proto";
-import "tajo_protos.proto";
-import "TajoIdProtos.proto";
-import "CatalogProtos.proto";
-import "PrimitiveProtos.proto";
-
-message QueryHeartbeat {
-  required ApplicationAttemptIdProto queryId = 1;
-  required string queryMasterHost = 2;
-  required int32 queryMasterPort = 3;
-  required int32 queryMasterClientPort = 4;
-  required QueryState state = 5;
-  optional string statusMessage = 6;
-}
-
-message QueryHeartbeatResponse {
-  message ResponseCommand {
-      required string command = 1;
-      repeated string params = 2;
-  }
-  required BoolProto heartbeatResult = 1;
-  optional ResponseCommand responseCommand = 3;
-}
-
-service QueryMasterManagerProtocolService {
-  rpc queryHeartbeat(QueryHeartbeat) returns (QueryHeartbeatResponse);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
deleted file mode 100644
index b6a0602..0000000
--- a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tajo.ipc";
-option java_outer_classname = "QueryMasterProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "yarn_protos.proto";
-import "tajo_protos.proto";
-import "TajoIdProtos.proto";
-import "CatalogProtos.proto";
-import "PrimitiveProtos.proto";
-
-message TaskStatusProto {
-  required QueryUnitAttemptIdProto id = 1;
-  required string workerName = 2;
-  required float progress = 3;
-  required TaskAttemptState state = 4;
-  optional StatSetProto stats = 5;
-  optional TableStatProto resultStats = 6;
-  repeated Partition partitions = 7;
-}
-
-message TaskCompletionReport {
-  required QueryUnitAttemptIdProto id = 1;
-  optional StatSetProto stats = 2;
-  optional TableStatProto resultStats = 3;
-  repeated Partition partitions = 4;
-}
-
-message TaskFatalErrorReport {
-  required QueryUnitAttemptIdProto id = 1;
-  optional string error_message = 2;
-}
-
-message QueryUnitRequestProto {
-    required QueryUnitAttemptIdProto id = 1;
-    repeated FragmentProto fragments = 2;
-    required string outputTable = 3;
-    required bool clusteredOutput = 4;
-    required string serializedData = 5;
-    optional bool interQuery = 6 [default = false];
-    repeated Fetch fetches = 7;
-    optional bool shouldDie = 8;
-}
-
-message Fetch {
-    required string name = 1;
-    required string urls = 2;
-}
-
-message QueryUnitResponseProto {
-    required string id = 1;
-    required QueryState status = 2;
-}
-
-message StatusReportProto {
-  required int64 timestamp = 1;
-  required string serverName = 2;
-  repeated TaskStatusProto status = 3;
-  repeated QueryUnitAttemptIdProto pings = 4;
-}
-
-message CommandRequestProto {
-    repeated Command command = 1;
-}
-
-message CommandResponseProto {
-}
-
-message Command {
-    required QueryUnitAttemptIdProto id = 1;
-    required CommandType type = 2;
-}
-
-enum CommandType {
-    PREPARE = 0;
-    LAUNCH = 1;
-    STOP = 2;
-    FINALIZE = 3;
-}
-
-message Partition {
-    required int32 partitionKey = 1;
-    optional string fileName = 2;
-}
-
-message ServerStatusProto {
-    message System {
-        required int32 availableProcessors = 1;
-        required int64 freeMemory = 2;
-        required int64 maxMemory = 3;
-        required int64 totalMemory = 4;
-    }
-    message Disk {
-        required string absolutePath = 1;
-        required int64 totalSpace = 2;
-        required int64 freeSpace = 3;
-        required int64 usableSpace = 4;
-    }
-    required System system = 1;
-    repeated Disk disk = 2;
-    required int32 taskNum = 3;
-}
-
-service QueryMasterProtocolService {
-  //from Worker
-  rpc getTask(ContainerIdProto) returns (QueryUnitRequestProto);
-  rpc statusUpdate (TaskStatusProto) returns (BoolProto);
-  rpc ping (QueryUnitAttemptIdProto) returns (BoolProto);
-  rpc fatalError(TaskFatalErrorReport) returns (BoolProto);
-  rpc done (TaskCompletionReport) returns (BoolProto);
-
-  //from QueryMasterManager
-  rpc executeQuery(StringProto) returns (BoolProto);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
index 04c67f2..a87c825 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
@@ -21,15 +21,18 @@ option java_outer_classname = "TajoIdProtos";
 option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
-import "yarn_protos.proto";
+message QueryIdProto {
+    required string id = 1;
+    required int32 seq = 2;
+}
 
-message SubQueryIdProto {
-    required ApplicationAttemptIdProto queryId = 1;
+message ExecutionBlockIdProto {
+    required QueryIdProto queryId = 1;
     required int32 id = 2;
 }
 
 message QueryUnitIdProto {
-    required SubQueryIdProto subQueryId = 1;
+    required ExecutionBlockIdProto executionBlockId = 1;
     required int32 id = 2;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
index ef7e711..26dbbed 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+//TajoClient -> TajoMaster Protocol
 option java_package = "org.apache.tajo.ipc";
 option java_outer_classname = "TajoMasterClientProtocol";
 option java_generic_services = true;
@@ -30,12 +31,12 @@ import "ClientProtos.proto";
 
 service TajoMasterClientProtocolService {
   rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
-  rpc submitQuery(QueryRequest) returns (SubmitQueryResponse);
+  rpc submitQuery(QueryRequest) returns (GetQueryStatusResponse);
   rpc updateQuery(QueryRequest) returns (UpdateQueryResponse);
   rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
   rpc getQueryList(GetQueryListRequest) returns (GetQueryListResponse);
   rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
-  rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+  rpc killQuery(QueryIdProto) returns (BoolProto);
   rpc getClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse);
   rpc existTable(StringProto) returns (BoolProto);
   rpc getTableList(GetTableListRequest) returns (GetTableListResponse);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
new file mode 100644
index 0000000..0153c8d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+//TajoWorker -> TajoMaster protocol
+
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoMasterProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+
+message ServerStatusProto {
+    message System {
+        required int32 availableProcessors = 1;
+        required int32 freeMemoryMB = 2;
+        required int32 maxMemoryMB = 3;
+        required int32 totalMemoryMB = 4;
+    }
+    message Disk {
+        required string absolutePath = 1;
+        required int64 totalSpace = 2;
+        required int64 freeSpace = 3;
+        required int64 usableSpace = 4;
+    }
+    required System system = 1;
+    required int32 diskSlots = 2;
+    repeated Disk disk = 3;
+    required int32 runningTaskNum = 4;
+}
+
+message TajoHeartbeat {
+  required string tajoWorkerHost = 1;
+  required int32 tajoWorkerPort = 2;
+  optional ServerStatusProto serverStatus = 3;
+  optional int32 tajoWorkerClientPort = 4;
+  optional QueryIdProto queryId = 5;
+  optional QueryState state = 6;
+  optional string statusMessage = 7;
+}
+
+message TajoHeartbeatResponse {
+  message ResponseCommand {
+      required string command = 1;
+      repeated string params = 2;
+  }
+  required BoolProto heartbeatResult = 1;
+  optional ResponseCommand responseCommand = 3;
+}
+
+message WorkerResourceAllocationRequest {
+    required ExecutionBlockIdProto executionBlockId = 1;
+    required int32 numWorks = 2;
+    required int32 memoryMBSlots = 3 ;
+    required int32 diskSlots = 4;
+}
+
+message WorkerResourceProto {
+    required string workerHostAndPort = 1;
+    required ExecutionBlockIdProto executionBlockId = 2;
+    required int32 memoryMBSlots = 3 ;
+    required int32 diskSlots = 4;
+}
+
+message WorkerResourceReleaseRequest {
+    repeated WorkerResourceProto workerResources = 1;
+}
+
+message WorkerResourceAllocationResponse {
+    required ExecutionBlockIdProto executionBlockId = 1;
+    repeated string allocatedWorks = 2;
+}
+
+service TajoMasterProtocolService {
+  rpc heartbeat(TajoHeartbeat) returns (TajoHeartbeatResponse);
+  rpc allocateWorkerResources(WorkerResourceAllocationRequest) returns (WorkerResourceAllocationResponse);
+  rpc releaseWorkerResource(WorkerResourceReleaseRequest) returns (BoolProto);
+  rpc stopQueryMaster(QueryIdProto) returns (BoolProto);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
new file mode 100644
index 0000000..88a2029
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// TajoMaster -> TajoWorker, TajoWorker(QueryMaster) <-> TajoWorker Protocol
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoWorkerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+
+message TaskStatusProto {
+  required QueryUnitAttemptIdProto id = 1;
+  required string workerName = 2;
+  required float progress = 3;
+  required TaskAttemptState state = 4;
+  optional StatSetProto stats = 5;
+  optional TableStatProto resultStats = 6;
+  repeated Partition partitions = 7;
+}
+
+message TaskCompletionReport {
+  required QueryUnitAttemptIdProto id = 1;
+  optional StatSetProto stats = 2;
+  optional TableStatProto resultStats = 3;
+  repeated Partition partitions = 4;
+}
+
+message TaskFatalErrorReport {
+  required QueryUnitAttemptIdProto id = 1;
+  optional string error_message = 2;
+}
+
+message QueryUnitRequestProto {
+    required QueryUnitAttemptIdProto id = 1;
+    repeated FragmentProto fragments = 2;
+    required string outputTable = 3;
+    required bool clusteredOutput = 4;
+    required string serializedData = 5;
+    optional bool interQuery = 6 [default = false];
+    repeated Fetch fetches = 7;
+    optional bool shouldDie = 8;
+}
+
+message Fetch {
+    required string name = 1;
+    required string urls = 2;
+}
+
+message QueryUnitResponseProto {
+    required string id = 1;
+    required QueryState status = 2;
+}
+
+message StatusReportProto {
+  required int64 timestamp = 1;
+  required string serverName = 2;
+  repeated TaskStatusProto status = 3;
+  repeated QueryUnitAttemptIdProto pings = 4;
+}
+
+message CommandRequestProto {
+    repeated Command command = 1;
+}
+
+message CommandResponseProto {
+}
+
+message Command {
+    required QueryUnitAttemptIdProto id = 1;
+    required CommandType type = 2;
+}
+
+enum CommandType {
+    PREPARE = 0;
+    LAUNCH = 1;
+    STOP = 2;
+    FINALIZE = 3;
+}
+
+message Partition {
+    required int32 partitionKey = 1;
+    optional string fileName = 2;
+}
+
+message QueryExecutionRequestProto {
+    required QueryIdProto queryId = 1;
+    required StringProto logicalPlanJson = 2;
+}
+
+message GetTaskRequestProto {
+    required ContainerIdProto containerId = 1;
+    required ExecutionBlockIdProto executionBlockId = 2;
+}
+
+message RunExecutionBlockRequestProto {
+    required string executionBlockId = 1;
+    required string queryMasterHost = 2;
+    required int32 queryMasterPort = 3;
+    required string nodeId = 4;
+    required string containerId = 5;
+    optional string queryOutputPath = 6;
+}
+
+service TajoWorkerProtocolService {
+  //from Worker
+  rpc getTask(GetTaskRequestProto) returns (QueryUnitRequestProto);
+  rpc statusUpdate (TaskStatusProto) returns (BoolProto);
+  rpc ping (QueryUnitAttemptIdProto) returns (BoolProto);
+  rpc fatalError(TaskFatalErrorReport) returns (BoolProto);
+  rpc done (TaskCompletionReport) returns (BoolProto);
+
+  //from TajoMaster's QueryJobManager
+  rpc executeQuery(QueryExecutionRequestProto) returns (BoolProto);
+
+  //from QueryMaster(Worker)
+  rpc executeExecutionBlock(RunExecutionBlockRequestProto) returns (BoolProto);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/log4j.properties b/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
index 2b42975..007c8f5 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
@@ -23,3 +23,6 @@ log4j.threshhold=INFO
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml b/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
index a1a111c..e8ad503 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
+++ b/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
@@ -43,4 +43,72 @@
   </property>
 
 
+
+  <property>
+    <name>tajo.master.clientservice.addr</name>
+    <value>127.0.0.1:9004</value>
+  </property>
+
+  <property>
+    <name>tajo.master.manager.addr</name>
+    <value>127.0.0.1:9005</value>
+    <description>rpc port for tajo worker</description>
+  </property>
+
+  <property>
+    <name>tajo.query.session.timeout</name>
+    <value>60000</value>
+    <description>ms</description>
+  </property>
+
+  <property>
+    <name>tajo.resource.manager</name>
+    <value>org.apache.tajo.master.rm.TajoWorkerResourceManager</value>
+    <description>This can be org.apache.tajo.master.rm.TajoWorkerResourceManager or org.apache.tajo.master.rm.YarnTajoResourceManager</description>
+  </property>
+
+  <property>
+    <name>tajo.querymaster.memoryMB</name>
+    <value>512</value>
+    <description>the memory slot size for a QeuryMaster</description>
+  </property>
+
+  <property>
+    <name>tajo.worker.slots.use.os.info</name>
+    <value>true</value>
+    <description>If true, Tajo system obtains the physical resource information from OS.
+                 If false, the physical resource information is obtained from the below configs.</description>
+  </property>
+
+  <!-- Default Node's Physical information -->
+  <!-- The below configs are used if tajo.worker.slots.use.os.info is set to true. -->
+  <property>
+    <name>tajo.worker.slots.os.memory.ratio</name>
+    <value>0.8f</value>
+    <description>The ratio of allocatable memory to the total system memory</description>
+  </property>
+
+  <property>
+    <name>tajo.worker.slots.memoryMB</name>
+    <value>2048</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>tajo.worker.slots.disk</name>
+    <value>2</value>
+    <description>The number of disks on a worker</description>
+  </property>
+
+  <property>
+    <name>tajo.worker.slots.disk.concurrency</name>
+    <value>4</value>
+    <description>the maximum concurrency number per disk slot</description>
+  </property>
+
+  <property>
+    <name>tajo.worker.slots.cpu.core</name>
+    <value>4</value>
+    <description>The number of CPU cores on a worker</description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/log4j.properties b/tajo-core/tajo-core-backend/src/test/java/log4j.properties
index c1ac487..749124c 100644
--- a/tajo-core/tajo-core-backend/src/test/java/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/test/java/log4j.properties
@@ -23,3 +23,6 @@ log4j.threshhold=ALL
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
index 5e8d11d..1667813 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -23,7 +23,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
@@ -83,7 +86,11 @@ public class LocalTajoTestingUtility {
   }
 
   public void shutdown() throws IOException {
-    client.close();
-    util.shutdownMiniCluster();
+    if(client != null) {
+      client.close();
+    }
+    if(util != null) {
+      util.shutdownMiniCluster();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
index 5b7267f..37e2721 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
@@ -110,9 +110,6 @@ public class MiniTajoYarnCluster extends MiniYARNCluster {
 
     conf.setInt("yarn.nodemanager.delete.debug-delay-sec", 600);
 
-    // Disable virtual memory constraints for containers
-    conf.setBoolean("yarn.nodemanager.vmem-check-enabled", false);
-
     super.init(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 32b1f56..041043d 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -27,9 +27,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -37,12 +36,17 @@ import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
 import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.TajoWorker;
 
 import java.io.*;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.URL;
 import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.UUID;
 
 public class TajoTestingCluster {
@@ -54,8 +58,9 @@ public class TajoTestingCluster {
   private MiniDFSCluster dfsCluster;
 	private MiniCatalogServer catalogServer;
 
-
   private TajoMaster tajoMaster;
+  private List<TajoWorker> tajoWorkers = new ArrayList<TajoWorker>();
+  private boolean standbyWorkerMode = false;
 
 	// If non-null, then already a cluster running.
 	private File clusterTestBuildDir = null;
@@ -73,7 +78,10 @@ public class TajoTestingCluster {
 	public static final String DEFAULT_TEST_DIRECTORY = "target/test-data";
 
 	public TajoTestingCluster() {
-		this.conf = new TajoConf();
+    this.conf = new TajoConf();
+    this.standbyWorkerMode =
+        this.conf.get("tajo.resource.manager", TajoWorkerResourceManager.class.getCanonicalName())
+            .indexOf(TajoWorkerResourceManager.class.getName()) >= 0;
 	}
 
 	public TajoConf getConfiguration() {
@@ -113,7 +121,7 @@ public class TajoTestingCluster {
 		String dirStr = getTestDir(randomStr).toString();
 		File dir = new File(dirStr).getAbsoluteFile();
 		// Have it cleaned up on exit
-		dir.deleteOnExit();
+		//dir.deleteOnExit();
 		return dir;
 	}
 
@@ -155,8 +163,7 @@ public class TajoTestingCluster {
     System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
         this.clusterTestBuildDir.toString());
 
-    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
-    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(new HdfsConfiguration(conf));
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
     builder.hosts(hosts);
     builder.numDataNodes(servers);
     builder.format(true);
@@ -210,7 +217,7 @@ public class TajoTestingCluster {
     catalogServer = new MiniCatalogServer(conf);
     CatalogServer catServer = catalogServer.getCatalogServer();
     InetSocketAddress sockAddr = catServer.getBindAddress();
-    c.setVar(ConfVars.CATALOG_ADDRESS, NetUtils.normalizeInetSocketAddress(sockAddr));
+    c.setVar(ConfVars.CATALOG_ADDRESS, NetUtils.getIpPortString(sockAddr));
 
     return this.catalogServer;
   }
@@ -232,11 +239,12 @@ public class TajoTestingCluster {
     TajoConf c = getConfiguration();
     c.setVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS, "localhost:0");
     c.setVar(ConfVars.CLIENT_SERVICE_ADDRESS, "localhost:0");
-    c.setVar(ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS, "localhost:0");
+    c.setVar(ConfVars.TAJO_MASTER_SERVICE_ADDRESS, "localhost:0");
 
     c.setVar(ConfVars.CATALOG_ADDRESS, "localhost:0");
     c.set(CatalogConstants.STORE_CLASS, "org.apache.tajo.catalog.store.MemStore");
     c.set(CatalogConstants.JDBC_URI, "jdbc:derby:target/test-data/tcat/db");
+
     LOG.info("derby repository is set to "+conf.get(CatalogConstants.JDBC_URI));
 
     if (!local) {
@@ -253,11 +261,38 @@ public class TajoTestingCluster {
 
     this.conf.setVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS, c.getVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS));
     this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS, c.getVar(ConfVars.CLIENT_SERVICE_ADDRESS));
+
+    InetSocketAddress tajoMasterAddress = tajoMaster.getContext().getTajoMasterService().getBindAddress();
+
+    this.conf.setVar(ConfVars.TAJO_MASTER_SERVICE_ADDRESS,
+        tajoMasterAddress.getHostName() + ":" + tajoMasterAddress.getPort());
+
     this.conf.setVar(ConfVars.CATALOG_ADDRESS, c.getVar(ConfVars.CATALOG_ADDRESS));
 
+    if(standbyWorkerMode) {
+      startTajoWorkers(numSlaves);
+    }
     LOG.info("Mini Tajo cluster is up");
   }
 
+  private void startTajoWorkers(int numSlaves) throws Exception {
+    for(int i = 0; i < 1; i++) {
+      TajoWorker tajoWorker = new TajoWorker("all");
+
+      TajoConf workerConf  = new TajoConf(this.conf);
+
+      workerConf.setInt("tajo.worker.info.port", 0);
+      workerConf.setInt("tajo.worker.client.rpc.port", 0);
+      workerConf.setInt("tajo.worker.manager.rpc.port", 0);
+      workerConf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname, 0);
+
+      tajoWorker.startWorker(workerConf, new String[]{"standby"});
+
+      LOG.info("=====> MiniTajoCluster Worker #" + (i + 1) + " started.");
+      tajoWorkers.add(tajoWorker);
+    }
+  }
+
   public void restartTajoCluster(int numSlaves) throws Exception {
     tajoMaster.stop();
     tajoMaster.start();
@@ -273,6 +308,10 @@ public class TajoTestingCluster {
     if(this.tajoMaster != null) {
       this.tajoMaster.stop();
     }
+    for(TajoWorker eachWorker: tajoWorkers) {
+      eachWorker.stopWorkerForce();
+    }
+    tajoWorkers.clear();
     this.tajoMaster= null;
   }
 
@@ -297,7 +336,8 @@ public class TajoTestingCluster {
    */
   public void startMiniCluster(final int numSlaves)
       throws Exception {
-    startMiniCluster(numSlaves, null);
+    String localHostName = InetAddress.getLocalHost().getHostName();
+    startMiniCluster(numSlaves, new String[] {localHostName});
   }
 
   public void startMiniCluster(final int numSlaves,
@@ -331,17 +371,19 @@ public class TajoTestingCluster {
     startMiniDFSCluster(numDataNodes, this.clusterTestBuildDir, dataNodeHosts);
     this.dfsCluster.waitClusterUp();
 
+    if(!standbyWorkerMode) {
+      startMiniYarnCluster();
+    }
+
+    startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, false);
+  }
+
+  private void startMiniYarnCluster() throws Exception {
     LOG.info("Starting up YARN cluster");
     // Scheduler properties required for YARN to work
     conf.set("yarn.scheduler.capacity.root.queues", "default");
     conf.set("yarn.scheduler.capacity.root.default.capacity", "100");
 
-    // fixed thread OOM
-    conf.setInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT, 2);
-    conf.setInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT, 2);
-    conf.setInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT, 2);
-    conf.setInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT, 2);
-
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 384);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 3000);
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, 1);
@@ -368,8 +410,6 @@ public class TajoTestingCluster {
       yarnCluster.getConfig().writeXml(os);
       os.close();
     }
-
-    startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, false);
   }
 
   public void startMiniClusterInLocal(final int numSlaves) throws Exception {
@@ -413,9 +453,12 @@ public class TajoTestingCluster {
     }
 
     if(this.clusterTestBuildDir != null && this.clusterTestBuildDir.exists()) {
-      LocalFileSystem localFS = LocalFileSystem.getLocal(conf);
-      localFS.delete(
-          new Path(clusterTestBuildDir.toString()), true);
+      if(!ShutdownHookManager.get().isShutdownInProgress()) {
+        //TODO clean test dir when ShutdownInProgress
+        LocalFileSystem localFS = LocalFileSystem.getLocal(conf);
+        localFS.delete(
+            new Path(clusterTestBuildDir.toString()), true);
+      }
       this.clusterTestBuildDir = null;
     }
 
@@ -457,6 +500,12 @@ public class TajoTestingCluster {
                               String query) throws Exception {
     TpchTestBase instance = TpchTestBase.getInstance();
     TajoTestingCluster util = instance.getTestingCluster();
+    while(true) {
+      if(util.getMaster().isMasterRunning()) {
+        break;
+      }
+      Thread.sleep(1000);
+    }
     TajoConf conf = util.getConfiguration();
     TajoClient client = new TajoClient(conf);
 
@@ -503,33 +552,4 @@ public class TajoTestingCluster {
       Closeables.closeQuietly(writer);
     }
   }
-
-
-	/**
-	 * @param args
-	 * @throws Exception
-	 */
-	public static void main(String[] args) throws Exception {
-		TajoTestingCluster cluster = new TajoTestingCluster();
-    File f = cluster.setupClusterTestBuildDir();
-    System.out.println("first setupClusterTestBuildDir: " + f);
-    f = cluster.setupClusterTestBuildDir();
-    System.out.println("second setupClusterTestBuildDir: " + f);
-    f = cluster.getTestDir();
-    System.out.println("getTestDir() after second: " + f);
-    f = cluster.getTestDir("abc");
-    System.out.println("getTestDir(\"abc\") after second: " + f);
-
-    cluster.initTestDir();
-    f = cluster.getTestDir();
-    System.out.println("getTestDir() after initTestDir: " + f);
-    f = cluster.getTestDir("abc");
-    System.out.println("getTestDir(\"abc\") after initTestDir: " + f);
-    f = cluster.setupClusterTestBuildDir();
-    System.out.println("setupClusterTestBuildDir() after initTestDir: " + f);
-
-    TajoTestingCluster cluster2 = new TajoTestingCluster();
-    File f2 = cluster2.setupClusterTestBuildDir();
-    System.out.println("first setupClusterTestBuildDir of cluster2: " + f2);
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
index b4d920f..7b82952 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
@@ -27,7 +27,6 @@ public class TestQueryIdFactory {
   
   @Before
   public void setup() {
-    QueryIdFactory.reset();
   }
 
   @Test
@@ -40,15 +39,15 @@ public class TestQueryIdFactory {
   @Test
   public void testNewSubQueryId() {
     QueryId qid = QueryIdFactory.newQueryId();
-    SubQueryId subqid1 = QueryIdFactory.newSubQueryId(qid);
-    SubQueryId subqid2 = QueryIdFactory.newSubQueryId(qid);
+    ExecutionBlockId subqid1 = QueryIdFactory.newExecutionBlockId(qid);
+    ExecutionBlockId subqid2 = QueryIdFactory.newExecutionBlockId(qid);
     assertTrue(subqid1.compareTo(subqid2) < 0);
   }
   
   @Test
   public void testNewQueryUnitId() {
     QueryId qid = QueryIdFactory.newQueryId();
-    SubQueryId subid = QueryIdFactory.newSubQueryId(qid);
+    ExecutionBlockId subid = QueryIdFactory.newExecutionBlockId(qid);
     QueryUnitId quid1 = QueryIdFactory.newQueryUnitId(subid);
     QueryUnitId quid2 = QueryIdFactory.newQueryUnitId(subid);
     assertTrue(quid1.compareTo(quid2) < 0);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
index 386fe02..1997159 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
@@ -20,8 +20,8 @@ package org.apache.tajo;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.junit.Test;
 import org.apache.tajo.util.TajoIdUtils;
+import org.junit.Test;
 
 import static org.junit.Assert.*;
 
@@ -31,10 +31,10 @@ public class TestTajoIds {
     long ts1 = 1315890136000l;
     long ts2 = 1315890136001l;
 
-    QueryId j1 = createQueryId(ts1, 2, 1);
-    QueryId j2 = createQueryId(ts1, 1, 2);
-    QueryId j3 = createQueryId(ts2, 1, 2);
-    QueryId j4 = createQueryId(ts1, 2, 1);
+    QueryId j1 = createQueryId(ts1, 2);
+    QueryId j2 = createQueryId(ts1, 1);
+    QueryId j3 = createQueryId(ts2, 1);
+    QueryId j4 = createQueryId(ts1, 2);
 
     assertTrue(j1.equals(j4));
     assertFalse(j1.equals(j2));
@@ -48,42 +48,42 @@ public class TestTajoIds {
     assertFalse(j1.hashCode() == j2.hashCode());
     assertFalse(j1.hashCode() == j3.hashCode());
 
-    QueryId j5 = createQueryId(ts1, 231415, 2);
-    assertEquals("q_" + ts1 + "_0002_000001", j1.toString());
-    assertEquals("q_" + ts1 + "_231415_000002", j5.toString());
+    QueryId j5 = createQueryId(ts1, 231415);
+    assertEquals("q_" + ts1 + "_0002", j1.toString());
+    assertEquals("q_" + ts1 + "_231415", j5.toString());
   }
 
   @Test
   public void testQueryIds() {
     long timeId = 1315890136000l;
     
-    QueryId queryId = createQueryId(timeId, 1, 1);
-    assertEquals("q_" + timeId + "_0001_000001", queryId.toString());
+    QueryId queryId = createQueryId(timeId, 1);
+    assertEquals("q_" + timeId + "_0001", queryId.toString());
     
-    SubQueryId subId = TajoIdUtils.newSubQueryId(queryId, 2);
-    assertEquals("sq_" + timeId +"_0001_000001_02", subId.toString());
+    ExecutionBlockId subId = QueryIdFactory.newExecutionBlockId(queryId, 2);
+    assertEquals("eb_" + timeId +"_0001_000002", subId.toString());
     
     QueryUnitId qId = new QueryUnitId(subId, 5);
-    assertEquals("t_" + timeId + "_0001_000001_02_000005", qId.toString());
+    assertEquals("t_" + timeId + "_0001_000002_000005", qId.toString());
 
     QueryUnitAttemptId attemptId = new QueryUnitAttemptId(qId, 4);
-    assertEquals("ta_" + timeId + "_0001_000001_02_000005_04", attemptId.toString());
+    assertEquals("ta_" + timeId + "_0001_000002_000005_04", attemptId.toString());
   }
 
   @Test
   public void testEqualsObject() {
     long timeId = System.currentTimeMillis();
     
-    QueryId queryId1 = createQueryId(timeId, 1, 1);
-    QueryId queryId2 = createQueryId(timeId, 2, 2);
+    QueryId queryId1 = createQueryId(timeId, 1);
+    QueryId queryId2 = createQueryId(timeId, 2);
     assertNotSame(queryId1, queryId2);    
-    QueryId queryId3 = createQueryId(timeId, 1, 1);
+    QueryId queryId3 = createQueryId(timeId, 1);
     assertEquals(queryId1, queryId3);
     
-    SubQueryId sid1 = TajoIdUtils.newSubQueryId(queryId1, 1);
-    SubQueryId sid2 = TajoIdUtils.newSubQueryId(queryId1, 2);    
+    ExecutionBlockId sid1 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+    ExecutionBlockId sid2 = QueryIdFactory.newExecutionBlockId(queryId1, 2);
     assertNotSame(sid1, sid2);
-    SubQueryId sid3 = TajoIdUtils.newSubQueryId(queryId1, 1);
+    ExecutionBlockId sid3 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
     assertEquals(sid1, sid3);
     
     QueryUnitId qid1 = new QueryUnitId(sid1, 9);
@@ -97,16 +97,16 @@ public class TestTajoIds {
   public void testCompareTo() {
     long time = System.currentTimeMillis();
     
-    QueryId queryId1 = createQueryId(time, 1, 1);
-    QueryId queryId2 = createQueryId(time, 2, 2);
-    QueryId queryId3 = createQueryId(time, 1, 1);
+    QueryId queryId1 = createQueryId(time, 1);
+    QueryId queryId2 = createQueryId(time, 2);
+    QueryId queryId3 = createQueryId(time, 1);
     assertEquals(-1, queryId1.compareTo(queryId2));
     assertEquals(1, queryId2.compareTo(queryId1));
     assertEquals(0, queryId3.compareTo(queryId1));
-    
-    SubQueryId sid1 = TajoIdUtils.newSubQueryId(queryId1, 1);
-    SubQueryId sid2 = TajoIdUtils.newSubQueryId(queryId1, 2);    
-    SubQueryId sid3 = TajoIdUtils.newSubQueryId(queryId1, 1);
+
+    ExecutionBlockId sid1 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+    ExecutionBlockId sid2 = QueryIdFactory.newExecutionBlockId(queryId1, 2);
+    ExecutionBlockId sid3 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
     assertEquals(-1, sid1.compareTo(sid2));
     assertEquals(1, sid2.compareTo(sid1));
     assertEquals(0, sid3.compareTo(sid1));
@@ -121,33 +121,33 @@ public class TestTajoIds {
   
   @Test
   public void testConstructFromString() {
-    QueryIdFactory.reset();
+//    QueryIdFactory.reset();
     QueryId qid1 = QueryIdFactory.newQueryId();
-    QueryId qid2 = TajoIdUtils.createQueryId(qid1.toString());
+    QueryId qid2 = TajoIdUtils.parseQueryId(qid1.toString());
     assertEquals(qid1, qid2);
-    
-    SubQueryId sub1 = QueryIdFactory.newSubQueryId(qid1);
-    SubQueryId sub2 = TajoIdUtils.newSubQueryId(sub1.toString());
+
+    ExecutionBlockId sub1 = QueryIdFactory.newExecutionBlockId(qid1);
+    ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
     assertEquals(sub1, sub2);
     
     QueryUnitId u1 = QueryIdFactory.newQueryUnitId(sub1);
-    QueryUnitId u2 = new QueryUnitId(u1.toString());
+    QueryUnitId u2 = new QueryUnitId(u1.getProto());
     assertEquals(u1, u2);
 
     QueryUnitAttemptId attempt1 = new QueryUnitAttemptId(u1, 1);
-    QueryUnitAttemptId attempt2 = new QueryUnitAttemptId(attempt1.toString());
+    QueryUnitAttemptId attempt2 = new QueryUnitAttemptId(attempt1.getProto());
     assertEquals(attempt1, attempt2);
   }
 
   @Test
   public void testConstructFromPB() {
-    QueryIdFactory.reset();
+//    QueryIdFactory.reset();
     QueryId qid1 = QueryIdFactory.newQueryId();
     QueryId qid2 = new QueryId(qid1.getProto());
     assertEquals(qid1, qid2);
 
-    SubQueryId sub1 = QueryIdFactory.newSubQueryId(qid1);
-    SubQueryId sub2 = new SubQueryId(sub1.getProto());
+    ExecutionBlockId sub1 = QueryIdFactory.newExecutionBlockId(qid1);
+    ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
     assertEquals(sub1, sub2);
 
     QueryUnitId u1 = QueryIdFactory.newQueryUnitId(sub1);
@@ -159,9 +159,9 @@ public class TestTajoIds {
     assertEquals(attempt1, attempt2);
   }
 
-  public static QueryId createQueryId(long timestamp, int id, int attemptId) {
+  public static QueryId createQueryId(long timestamp, int id) {
     ApplicationId appId = BuilderUtils.newApplicationId(timestamp, id);
 
-    return TajoIdUtils.createQueryId(appId, attemptId);
+    return QueryIdFactory.newQueryId(appId.toString());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
index c761103..ad3d676 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
@@ -49,7 +49,6 @@ public class TpchTestBase {
     try {
       testBase = new TpchTestBase();
       testBase.setUp();
-      Runtime.getRuntime().addShutdownHook(new ShutdownHook());
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
     }
@@ -107,19 +106,11 @@ public class TpchTestBase {
     return util.getTestingCluster();
   }
 
-  public static class ShutdownHook extends Thread {
-
-    @Override
-    public void run() {
-      try {
-        testBase.tearDown();
-      } catch (IOException e) {
-        LOG.error(e);
-      }
+  public void tearDown() throws IOException {
+    try {
+      Thread.sleep(2000);
+    } catch (InterruptedException e) {
     }
-  }
-
-  private void tearDown() throws IOException {
     util.shutdown();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
index cc75726..ede73c5 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
@@ -136,7 +136,6 @@ public class TestGlobalQueryPlanner {
       catalog.addTable(desc);
     }
 
-    QueryIdFactory.reset();
     queryId = QueryIdFactory.newQueryId();
     dispatcher.stop();
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
index 4455763..c665b44 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
@@ -16,9 +16,6 @@
  * limitations under the License.
  */
 
-/**
- * 
- */
 package org.apache.tajo.engine.planner.global;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -128,11 +125,11 @@ public class TestGlobalQueryOptimizer {
       catalog.addTable(desc);
     }
 
-    QueryIdFactory.reset();
+    //QueryIdFactory.reset();
     queryId = QueryIdFactory.newQueryId();
     optimizer = new GlobalOptimizer();
   }
-  
+
   @AfterClass
   public static void terminate() throws IOException {
     util.shutdownCatalogCluster();
@@ -147,7 +144,7 @@ public class TestGlobalQueryOptimizer {
 
     MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
     globalPlan = optimizer.optimize(globalPlan);
-    
+
     ExecutionBlock unit = globalPlan.getRoot();
     StoreTableNode store = unit.getStoreTableNode();
     assertEquals(NodeType.PROJECTION, store.getChild().getType());
@@ -156,14 +153,14 @@ public class TestGlobalQueryOptimizer {
     SortNode sort = (SortNode) proj.getChild();
     assertEquals(NodeType.SCAN, sort.getChild().getType());
     ScanNode scan = (ScanNode) sort.getChild();
-    
+
     assertTrue(unit.hasChildBlock());
     unit = unit.getChildBlock(scan);
     store = unit.getStoreTableNode();
     assertEquals(NodeType.SORT, store.getChild().getType());
     sort = (SortNode) store.getChild();
     assertEquals(NodeType.JOIN, sort.getChild().getType());
-    
+
     assertTrue(unit.hasChildBlock());
     for (ScanNode prevscan : unit.getScanNodes()) {
       ExecutionBlock prev = unit.getChildBlock(prevscan);
@@ -171,4 +168,4 @@ public class TestGlobalQueryOptimizer {
       assertEquals(NodeType.SCAN, store.getChild().getType());
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index dbb3862..75e3b1e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -25,7 +25,6 @@ import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.TaskAttemptContext;
@@ -78,7 +77,6 @@ public class TestPhysicalPlanner {
 
   @BeforeClass
   public static void setUp() throws Exception {
-    QueryIdFactory.reset();
     util = new TajoTestingCluster();
 
     util.startCatalogCluster();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
index a8924dd..843df23 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
@@ -18,14 +18,13 @@
 
 package org.apache.tajo.master;
 
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.TestTajoIds;
 import org.apache.tajo.master.ExecutionBlock.PartitionType;
 import org.apache.tajo.master.querymaster.QueryUnit;
 import org.apache.tajo.master.querymaster.Repartitioner;
 import org.apache.tajo.util.TUtil;
-import org.apache.tajo.util.TajoIdUtils;
 import org.jboss.netty.handler.codec.http.QueryStringDecoder;
 import org.junit.Test;
 
@@ -37,10 +36,10 @@ import static junit.framework.Assert.assertEquals;
 public class TestRepartitioner {
   @Test
   public void testCreateHashFetchURL() throws Exception {
-    QueryId q1 = TestTajoIds.createQueryId(1315890136000l, 2, 1);
+    QueryId q1 = TestTajoIds.createQueryId(1315890136000l, 2);
     String hostName = "tajo1";
     int port = 1234;
-    SubQueryId sid = TajoIdUtils.createSubQueryId(q1, 2);
+    ExecutionBlockId sid = new ExecutionBlockId(q1, 2);
     int partitionId = 2;
 
     List<QueryUnit.IntermediateEntry> intermediateEntries = TUtil.newList();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
index 05a269e..952cb0f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
@@ -25,31 +25,27 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.tajo.QueryConf;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.TestTajoIds;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService;
 import org.apache.tajo.rpc.ProtoAsyncRpcClient;
-import org.apache.tajo.util.TajoIdUtils;
 
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 public class TaskRunnerTest {
   long ts1 = 1315890136000l;
-  QueryId q1 = TestTajoIds.createQueryId(ts1, 2, 5);
-  SubQueryId sq1 = TajoIdUtils.createSubQueryId(q1, 5);
+  QueryId q1 = TestTajoIds.createQueryId(ts1, 2);
+  ExecutionBlockId sq1 = QueryIdFactory.newExecutionBlockId(q1, 5);
 
   //@Test
   public void testInit() throws Exception {
     ProtoAsyncRpcClient mockClient = mock(ProtoAsyncRpcClient.class);
     mockClient.close();
 
-    QueryMasterProtocolService.Interface mockMaster =
-        mock(QueryMasterProtocolService.Interface.class);
+    TajoWorkerProtocolService.Interface mockMaster =
+        mock(TajoWorkerProtocolService.Interface.class);
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
-        q1.getApplicationId(), q1.getAttemptId());
+        BuilderUtils.newApplicationId(Integer.parseInt(q1.getId()), q1.getSeq()), 1);
     ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 1);
 
     NodeId nodeId = RecordFactoryProvider.getRecordFactory(null).

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 09ab483..cf1e9ae 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -23,7 +23,6 @@ import com.google.common.collect.Maps;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoTestingCluster;
 import org.apache.tajo.TaskAttemptContext;
 import org.apache.tajo.algebra.Expr;
@@ -73,7 +72,6 @@ public class TestRangeRetrieverHandler {
 
   @Before
   public void setUp() throws Exception {
-    QueryIdFactory.reset();
     util = new TajoTestingCluster();
     conf = util.getConfiguration();
     testDir = CommonTestingUtil.getTestDir("target/test-data/TestRangeRetrieverHandler");

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
index 6e6fdae..b70dda2 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
@@ -19,15 +19,15 @@
 package org.apache.tajo.worker.dataserver;
 
 import org.apache.hadoop.net.NetUtils;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.worker.InterDataRetriever;
 import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
 import org.apache.tajo.worker.dataserver.retriever.DirectoryRetriever;
+import org.junit.Before;
+import org.junit.Test;
 
 import java.io.*;
 import java.net.InetSocketAddress;
@@ -77,8 +77,7 @@ public class TestHttpDataServer {
   
   @Test
   public final void testInterDataRetriver() throws Exception {
-    QueryIdFactory.reset();
-    SubQueryId schid = QueryIdFactory.newSubQueryId(
+    ExecutionBlockId schid = QueryIdFactory.newExecutionBlockId(
             QueryIdFactory.newQueryId());
     QueryUnitId qid1 = QueryIdFactory.newQueryUnitId(schid);
     QueryUnitId qid2 = QueryIdFactory.newQueryUnitId(schid);
@@ -119,8 +118,7 @@ public class TestHttpDataServer {
   
   @Test(expected = FileNotFoundException.class)
   public final void testNoSuchFile() throws Exception {
-    QueryIdFactory.reset();
-    SubQueryId schid = QueryIdFactory.newSubQueryId(
+    ExecutionBlockId schid = QueryIdFactory.newExecutionBlockId(
             QueryIdFactory.newQueryId());
     QueryUnitId qid1 = QueryIdFactory.newQueryUnitId(schid);
     QueryUnitId qid2 = QueryIdFactory.newQueryUnitId(schid);


[6/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java
deleted file mode 100644
index e6d4c56..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
-import org.apache.tajo.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
-import org.apache.tajo.master.querymaster.QueryMaster;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
-import org.apache.tajo.worker.TaskRunner;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Vector;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-public class TaskRunnerLauncherImpl extends AbstractService implements TaskRunnerLauncher {
-
-  /** Class Logger */
-  private static final Log LOG = LogFactory.getLog(TaskRunnerLauncherImpl.class);
-  private QueryContext context;
-  private final String queryMasterHost;
-  private final int queryMasterPort;
-
-  // For ContainerLauncherSpec
-  private ContainerLaunchContext commonContainerSpec = null;
-
-  /** for launching TaskRunners in parallel */
-  private final ExecutorService executorService;
-
-  public TaskRunnerLauncherImpl(QueryContext context) {
-    super(TaskRunnerLauncherImpl.class.getName());
-    this.context = context;
-    queryMasterHost = context.getQueryMasterServiceAddress().getHostName();
-    queryMasterPort = context.getQueryMasterServiceAddress().getPort();
-    executorService = Executors.newFixedThreadPool(
-        context.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
-  }
-
-  public void start() {
-    super.start();
-  }
-
-  public void stop() {
-    executorService.shutdownNow();
-    Map<ContainerId, ContainerProxy> containers = context.getContainers();
-    for(ContainerProxy eachProxy: containers.values()) {
-      try {
-        eachProxy.kill();
-      } catch (Exception e) {
-      }
-    }
-    super.stop();
-  }
-
-  @Override
-  public void handle(TaskRunnerGroupEvent event) {
-    if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) {
-     launchTaskRunners(event.subQueryId, event.getContainers());
-    } else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
-      killTaskRunners(event.getContainers());
-    }
-  }
-
-  private void launchTaskRunners(SubQueryId subQueryId, Collection<Container> containers) {
-    commonContainerSpec = ContainerProxy.createCommonContainerLaunchContext(getConfig(), subQueryId.toString(), false);
-    for (Container container : containers) {
-      final ContainerProxy proxy =
-          new TaskRunnerContainerProxy(context, getConfig(), context.getYarnRPC(), container, subQueryId);
-      executorService.submit(new LaunchRunner(container.getId(), proxy));
-    }
-  }
-
-  private class LaunchRunner implements Runnable {
-    private final ContainerProxy proxy;
-    private final ContainerId id;
-
-    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
-      this.proxy = proxy;
-      this.id = id;
-    }
-    @Override
-    public void run() {
-      proxy.launch(commonContainerSpec);
-      LOG.info("ContainerProxy started:" + id);
-    }
-  }
-
-  private void killTaskRunners(Collection<Container> containers) {
-    for (Container container : containers) {
-      final ContainerProxy proxy = context.getContainer(container.getId());
-      executorService.submit(new KillRunner(container.getId(), proxy));
-    }
-  }
-
-  private class KillRunner implements Runnable {
-    private final ContainerProxy proxy;
-    private final ContainerId id;
-    public KillRunner(ContainerId id, ContainerProxy proxy) {
-      this.id = id;
-      this.proxy = proxy;
-    }
-
-    @Override
-    public void run() {
-      proxy.kill();
-      LOG.info("ContainerProxy killed:" + id);
-    }
-  }
-
-  public class TaskRunnerContainerProxy extends ContainerProxy {
-    private final SubQueryId subQueryId;
-
-    public TaskRunnerContainerProxy(QueryMaster.QueryContext context, Configuration conf, YarnRPC yarnRPC,
-                                    Container container, SubQueryId subQueryId) {
-      super(context, conf, yarnRPC, container);
-      this.subQueryId = subQueryId;
-    }
-
-    @Override
-    protected void containerStarted() {
-      context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
-    }
-
-    @Override
-    protected String getId() {
-      return subQueryId.toString();
-    }
-
-    @Override
-    protected String getRunnerClass() {
-      return TaskRunner.class.getCanonicalName();
-    }
-
-    @Override
-    protected Vector<CharSequence> getTaskParams() {
-      Vector<CharSequence> taskParams = new Vector<CharSequence>();
-      taskParams.add(queryMasterHost); // queryMaster hostname
-      taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
-
-      return taskParams;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
index c9702b4..651f9c0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
@@ -23,28 +23,25 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.engine.planner.logical.ScanNode;
 import org.apache.tajo.engine.query.QueryUnitRequestImpl;
-import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
 import org.apache.tajo.master.event.TaskAttemptAssignedEvent;
 import org.apache.tajo.master.event.TaskRequestEvent;
-import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
 import org.apache.tajo.master.event.TaskScheduleEvent;
 import org.apache.tajo.master.event.TaskSchedulerEvent;
 import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
 import org.apache.tajo.master.querymaster.QueryUnit;
 import org.apache.tajo.storage.Fragment;
 import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
 
 import java.net.URI;
 import java.util.*;
@@ -55,8 +52,8 @@ public class TaskSchedulerImpl extends AbstractService
     implements TaskScheduler {
   private static final Log LOG = LogFactory.getLog(TaskScheduleEvent.class);
 
-  private final QueryContext context;
-  private AsyncDispatcher dispatcher;
+  private final QueryMasterTask.QueryContext context;
+  private TajoAsyncDispatcher dispatcher;
 
   private Thread eventHandlingThread;
   private Thread schedulingThread;
@@ -72,21 +69,22 @@ public class TaskSchedulerImpl extends AbstractService
   private int rackLocalAssigned = 0;
   private int totalAssigned = 0;
 
-  public TaskSchedulerImpl(QueryContext context) {
+  public TaskSchedulerImpl(QueryMasterTask.QueryContext context) {
     super(TaskSchedulerImpl.class.getName());
     this.context = context;
     this.dispatcher = context.getDispatcher();
   }
 
+  @Override
   public void init(Configuration conf) {
 
     scheduledRequests = new ScheduledRequests();
     taskRequests  = new TaskRequests();
-    dispatcher.register(TaskRequestEventType.class, taskRequests);
 
     super.init(conf);
   }
 
+  @Override
   public void start() {
     LOG.info("Start TaskScheduler");
     this.eventHandlingThread = new Thread() {
@@ -113,13 +111,14 @@ public class TaskSchedulerImpl extends AbstractService
 
         while(!stopEventHandling && !Thread.currentThread().isInterrupted()) {
           try {
-            Thread.sleep(100);
+            Thread.sleep(1000);
           } catch (InterruptedException e) {
             break;
           }
 
           schedule();
         }
+        //req.getCallback().run(stopTaskRunnerReq);
         LOG.info("TaskScheduler schedulingThread stopped");
       }
     };
@@ -128,15 +127,15 @@ public class TaskSchedulerImpl extends AbstractService
     super.start();
   }
 
-  private static final QueryUnitAttemptId NULL_ID;
-  private static final QueryMasterProtocol.QueryUnitRequestProto stopTaskRunnerReq;
+  private static final QueryUnitAttemptId NULL_ATTEMPT_ID;
+  public static final TajoWorkerProtocol.QueryUnitRequestProto stopTaskRunnerReq;
   static {
-    SubQueryId nullSubQuery =
-        QueryIdFactory.newSubQueryId(TajoIdUtils.NullQueryId);
-    NULL_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
+    ExecutionBlockId nullSubQuery = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+    NULL_ATTEMPT_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
 
-    QueryMasterProtocol.QueryUnitRequestProto.Builder builder = QueryMasterProtocol.QueryUnitRequestProto.newBuilder();
-    builder.setId(NULL_ID.getProto());
+    TajoWorkerProtocol.QueryUnitRequestProto.Builder builder =
+        TajoWorkerProtocol.QueryUnitRequestProto.newBuilder();
+    builder.setId(NULL_ATTEMPT_ID.getProto());
     builder.setShouldDie(true);
     builder.setOutputTable("");
     builder.setSerializedData("");
@@ -144,7 +143,7 @@ public class TaskSchedulerImpl extends AbstractService
     stopTaskRunnerReq = builder.build();
   }
 
-
+  @Override
   public void stop() {
     stopEventHandling = true;
     eventHandlingThread.interrupt();
@@ -205,12 +204,12 @@ public class TaskSchedulerImpl extends AbstractService
   public void handle(TaskSchedulerEvent event) {
     int qSize = eventQueue.size();
     if (qSize != 0 && qSize % 1000 == 0) {
-      LOG.info("Size of event-queue in RMContainerAllocator is " + qSize);
+      LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
     }
     int remCapacity = eventQueue.remainingCapacity();
     if (remCapacity < 1000) {
       LOG.warn("Very low remaining capacity in the event-queue "
-          + "of RMContainerAllocator: " + remCapacity);
+          + "of YarnRMContainerAllocator: " + remCapacity);
     }
 
     try {
@@ -220,20 +219,29 @@ public class TaskSchedulerImpl extends AbstractService
     }
   }
 
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    taskRequests.handle(event);
+  }
+
   private class TaskRequests implements EventHandler<TaskRequestEvent> {
     private final LinkedBlockingQueue<TaskRequestEvent> taskRequestQueue =
         new LinkedBlockingQueue<TaskRequestEvent>();
 
     @Override
     public void handle(TaskRequestEvent event) {
+      LOG.info("====>TaskRequest:" + event.getContainerId() + "," + event.getExecutionBlockId());
+      if(stopEventHandling) {
+        event.getCallback().run(stopTaskRunnerReq);
+        return;
+      }
       int qSize = taskRequestQueue.size();
       if (qSize != 0 && qSize % 1000 == 0) {
-        LOG.info("Size of event-queue in RMContainerAllocator is " + qSize);
+        LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
       }
       int remCapacity = taskRequestQueue.remainingCapacity();
       if (remCapacity < 1000) {
         LOG.warn("Very low remaining capacity in the event-queue "
-            + "of RMContainerAllocator: " + remCapacity);
+            + "of YarnRMContainerAllocator: " + remCapacity);
       }
 
       taskRequestQueue.add(event);
@@ -380,15 +388,16 @@ public class TaskSchedulerImpl extends AbstractService
       return nonLeafTasks.size();
     }
 
-    public Set<QueryUnitAttemptId> AssignedRequest = new HashSet<QueryUnitAttemptId>();
+    public Set<QueryUnitAttemptId> assignedRequest = new HashSet<QueryUnitAttemptId>();
+
     public void assignToLeafTasks(List<TaskRequestEvent> taskRequests) {
       Iterator<TaskRequestEvent> it = taskRequests.iterator();
-      LOG.info("Got task requests " + taskRequests.size());
 
       TaskRequestEvent taskRequest;
       while (it.hasNext() && leafTasks.size() > 0) {
         taskRequest = it.next();
-        ContainerProxy container = context.getContainer(taskRequest.getContainerId());
+        LOG.info("====> assignToLeafTasks: " + taskRequest.getExecutionBlockId());
+        ContainerProxy container = context.getResourceAllocator().getContainer(taskRequest.getContainerId());
         String host = container.getTaskHostName();
 
         QueryUnitAttemptId attemptId = null;
@@ -443,7 +452,7 @@ public class TaskSchedulerImpl extends AbstractService
 
         if (attemptId != null) {
           QueryUnit task = context.getQuery()
-              .getSubQuery(attemptId.getSubQueryId()).getQueryUnit(attemptId.getQueryUnitId());
+              .getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId()).getQueryUnit(attemptId.getQueryUnitId());
           QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
               attemptId,
               new ArrayList<Fragment>(task.getAllFragments()),
@@ -457,7 +466,7 @@ public class TaskSchedulerImpl extends AbstractService
           context.getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
               taskRequest.getContainerId(),
               host, container.getTaskPort()));
-          AssignedRequest.add(attemptId);
+          assignedRequest.add(attemptId);
 
           totalAssigned++;
           taskRequest.getCallback().run(taskAssign.getProto());
@@ -476,6 +485,7 @@ public class TaskSchedulerImpl extends AbstractService
       TaskRequestEvent taskRequest;
       while (it.hasNext()) {
         taskRequest = it.next();
+        LOG.info("====> assignToNonLeafTasks: " + taskRequest.getExecutionBlockId());
 
         QueryUnitAttemptId attemptId;
         // random allocation
@@ -485,7 +495,8 @@ public class TaskSchedulerImpl extends AbstractService
           LOG.debug("Assigned based on * match");
 
           QueryUnit task;
-          task = context.getSubQuery(attemptId.getSubQueryId()).getQueryUnit(attemptId.getQueryUnitId());
+          task = context.getSubQuery(
+              attemptId.getQueryUnitId().getExecutionBlockId()).getQueryUnit(attemptId.getQueryUnitId());
           QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
               attemptId,
               Lists.newArrayList(task.getAllFragments()),
@@ -504,7 +515,7 @@ public class TaskSchedulerImpl extends AbstractService
             }
           }
 
-          ContainerProxy container = context.getContainer(
+          ContainerProxy container = context.getResourceAllocator().getContainer(
               taskRequest.getContainerId());
           context.getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
               taskRequest.getContainerId(), container.getTaskHostName(), container.getTaskPort()));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
new file mode 100644
index 0000000..1e650e1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
@@ -0,0 +1,446 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.ProtoUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.event.QueryEvent;
+import org.apache.tajo.master.event.QueryEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.pullserver.PullServerAuxService;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.util.*;
+
+public class YarnContainerProxy extends ContainerProxy {
+  private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+
+  protected final YarnRPC yarnRPC;
+  final protected String containerMgrAddress;
+  protected ContainerToken containerToken;
+
+  public YarnContainerProxy(QueryMasterTask.QueryContext context, Configuration conf, YarnRPC yarnRPC,
+                                  Container container, ExecutionBlockId executionBlockId) {
+    super(context, conf, executionBlockId, container);
+    this.yarnRPC = yarnRPC;
+
+    NodeId nodeId = container.getNodeId();
+    this.containerMgrAddress = nodeId.getHost() + ":" + nodeId.getPort();
+    this.containerToken = container.getContainerToken();
+  }
+
+  protected ContainerManager getCMProxy(ContainerId containerID,
+                                        final String containerManagerBindAddr,
+                                        ContainerToken containerToken)
+      throws IOException {
+    String [] hosts = containerManagerBindAddr.split(":");
+    final InetSocketAddress cmAddr =
+        new InetSocketAddress(hosts[0], Integer.parseInt(hosts[1]));
+    UserGroupInformation user = UserGroupInformation.getCurrentUser();
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      Token<ContainerTokenIdentifier> token =
+          ProtoUtils.convertFromProtoFormat(containerToken, cmAddr);
+      // the user in createRemoteUser in this context has to be ContainerID
+      user = UserGroupInformation.createRemoteUser(containerID.toString());
+      user.addToken(token);
+    }
+
+    ContainerManager proxy = user.doAs(new PrivilegedAction<ContainerManager>() {
+      @Override
+      public ContainerManager run() {
+        return (ContainerManager) yarnRPC.getProxy(ContainerManager.class,
+            cmAddr, conf);
+      }
+    });
+
+    return proxy;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public synchronized void launch(ContainerLaunchContext commonContainerLaunchContext) {
+    LOG.info("Launching Container with Id: " + containerID);
+    if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
+      state = ContainerState.DONE;
+      LOG.error("Container (" + containerID + " was killed before it was launched");
+      return;
+    }
+
+    ContainerManager proxy = null;
+    try {
+
+      proxy = getCMProxy(containerID, containerMgrAddress,
+          containerToken);
+
+      // Construct the actual Container
+      ContainerLaunchContext containerLaunchContext = createContainerLaunchContext(commonContainerLaunchContext);
+
+      // Now launch the actual container
+      StartContainerRequest startRequest = Records
+          .newRecord(StartContainerRequest.class);
+      startRequest.setContainerLaunchContext(containerLaunchContext);
+      StartContainerResponse response = proxy.startContainer(startRequest);
+
+      ByteBuffer portInfo = response
+          .getServiceResponse(PullServerAuxService.PULLSERVER_SERVICEID);
+
+      if(portInfo != null) {
+        port = PullServerAuxService.deserializeMetaData(portInfo);
+      }
+
+      LOG.info("PullServer port returned by ContainerManager for "
+          + containerID + " : " + port);
+
+      if(port < 0) {
+        this.state = ContainerState.FAILED;
+        throw new IllegalStateException("Invalid shuffle port number "
+            + port + " returned for " + containerID);
+      }
+
+      context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+
+      this.state = ContainerState.RUNNING;
+      this.hostName = containerMgrAddress.split(":")[0];
+      context.getResourceAllocator().addContainer(containerID, this);
+    } catch (Throwable t) {
+      String message = "Container launch failed for " + containerID + " : "
+          + StringUtils.stringifyException(t);
+      this.state = ContainerState.FAILED;
+      LOG.error(message);
+    } finally {
+      if (proxy != null) {
+        yarnRPC.stopProxy(proxy, conf);
+      }
+    }
+  }
+
+
+  public ContainerLaunchContext createContainerLaunchContext(ContainerLaunchContext commonContainerLaunchContext) {
+    // Setup environment by cloning from common env.
+    Map<String, String> env = commonContainerLaunchContext.getEnvironment();
+    Map<String, String> myEnv = new HashMap<String, String>(env.size());
+    myEnv.putAll(env);
+
+    // Duplicate the ByteBuffers for access by multiple containers.
+    Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
+    for (Map.Entry<String, ByteBuffer> entry : commonContainerLaunchContext.getServiceData().entrySet()) {
+      myServiceData.put(entry.getKey(), entry.getValue().duplicate());
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the local resources
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the necessary command to execute the application master
+    Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+    // Set java executable command
+    //LOG.info("Setting up app master command");
+    vargs.add("${JAVA_HOME}" + "/bin/java");
+    // Set Xmx based on am memory size
+    vargs.add("-Xmx2000m");
+    // Set Remote Debugging
+    //if (!context.getQuery().getSubQuery(event.getExecutionBlockId()).isLeafQuery()) {
+    //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
+    //}
+    // Set class name
+    //vargs.add(getRunnerClass());
+    vargs.add(TajoWorker.class.getCanonicalName());
+    vargs.add("tr");     //workerMode
+    vargs.add(getId()); // subqueryId
+    vargs.add(containerMgrAddress); // nodeId
+    vargs.add(containerID.toString()); // containerId
+    Vector<CharSequence> taskParams = getTaskParams();
+    if(taskParams != null) {
+      vargs.addAll(taskParams);
+    }
+
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+    // Get final commmand
+    StringBuilder command = new StringBuilder();
+    for (CharSequence str : vargs) {
+      command.append(str).append(" ");
+    }
+
+    LOG.info("Completed setting up TaskRunner command " + command.toString());
+    List<String> commands = new ArrayList<String>();
+    commands.add(command.toString());
+
+    return BuilderUtils.newContainerLaunchContext(containerID, commonContainerLaunchContext.getUser(),
+        container.getResource(), commonContainerLaunchContext.getLocalResources(), myEnv, commands,
+        myServiceData, null, new HashMap<ApplicationAccessType, String>());
+  }
+
+  public static ContainerLaunchContext createCommonContainerLaunchContext(Configuration config,
+                                                                          String queryId, boolean isMaster) {
+    TajoConf conf = (TajoConf)config;
+
+    ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
+    try {
+      ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the env variables to be setup
+    ////////////////////////////////////////////////////////////////////////////
+    LOG.info("Set the environment for the application master");
+
+    Map<String, String> environment = new HashMap<String, String>();
+    //String initialClassPath = getInitialClasspath(conf);
+    environment.put(ApplicationConstants.Environment.SHELL.name(), "/bin/bash");
+    if(System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()) != null) {
+      environment.put(ApplicationConstants.Environment.JAVA_HOME.name(), System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()));
+    }
+
+    // TODO - to be improved with org.apache.tajo.sh shell script
+    Properties prop = System.getProperties();
+
+    if (prop.getProperty("tajo.test", "FALSE").equalsIgnoreCase("TRUE") ||
+        (System.getenv("tajo.test") != null && System.getenv("tajo.test").equalsIgnoreCase("TRUE"))) {
+      LOG.info("=========> tajo.test is TRUE");
+      environment.put(ApplicationConstants.Environment.CLASSPATH.name(), prop.getProperty(
+          "java.class.path", null));
+      environment.put("tajo.test", "TRUE");
+    } else {
+      // Add AppMaster.jar location to classpath
+      // At some point we should not be required to add
+      // the hadoop specific classpaths to the env.
+      // It should be provided out of the box.
+      // For now setting all required classpaths including
+      // the classpath to "." for the application jar
+      StringBuilder classPathEnv = new StringBuilder("./");
+      //for (String c : conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+      for (String c : YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH) {
+        classPathEnv.append(':');
+        classPathEnv.append(c.trim());
+      }
+
+      classPathEnv.append(":" + System.getenv("TAJO_BASE_CLASSPATH"));
+      classPathEnv.append(":./log4j.properties:./*");
+      if(System.getenv("HADOOP_HOME") != null) {
+        environment.put("HADOOP_HOME", System.getenv("HADOOP_HOME"));
+        environment.put(
+            ApplicationConstants.Environment.HADOOP_COMMON_HOME.name(),
+            System.getenv("HADOOP_HOME"));
+        environment.put(
+            ApplicationConstants.Environment.HADOOP_HDFS_HOME.name(),
+            System.getenv("HADOOP_HOME"));
+        environment.put(
+            ApplicationConstants.Environment.HADOOP_YARN_HOME.name(),
+            System.getenv("HADOOP_HOME"));
+      }
+
+      if(System.getenv("TAJO_BASE_CLASSPATH") != null) {
+        environment.put("TAJO_BASE_CLASSPATH", System.getenv("TAJO_BASE_CLASSPATH"));
+      }
+      environment.put(ApplicationConstants.Environment.CLASSPATH.name(), classPathEnv.toString());
+    }
+
+    ctx.setEnvironment(environment);
+
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("=================================================");
+      for(Map.Entry<String, String> entry: environment.entrySet()) {
+        LOG.debug(entry.getKey() + "=" + entry.getValue());
+      }
+      LOG.debug("=================================================");
+    }
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the local resources
+    ////////////////////////////////////////////////////////////////////////////
+    Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
+    FileSystem fs = null;
+
+    LOG.info("defaultFS: " + conf.get("fs.default.name"));
+    LOG.info("defaultFS: " + conf.get("fs.defaultFS"));
+    try {
+      fs = FileSystem.get(conf);
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    FileContext fsCtx = null;
+    try {
+      fsCtx = FileContext.getFileContext(conf);
+    } catch (UnsupportedFileSystemException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    LOG.info("Writing a QueryConf to HDFS and add to local environment");
+    //Path queryConfPath = new Path(fs.getHomeDirectory(), QueryConf.FILENAME);
+    try {
+      //writeConf(conf, queryConfPath);
+      // TODO move to tajo temp
+      Path warehousePath = new Path(conf.getVar(TajoConf.ConfVars.ROOT_DIR), TajoConstants.WAREHOUSE_DIR);
+      Path queryConfPath = new Path(warehousePath, queryId);
+      if(isMaster) {
+        queryConfPath = new Path(queryConfPath, QueryConf.QUERY_MASTER_FILENAME);
+      } else {
+        queryConfPath = new Path(queryConfPath, QueryConf.FILENAME);
+      }
+
+      if(!fs.exists(queryConfPath)){
+        writeConf(conf, queryConfPath);
+      } else {
+        LOG.warn("QueryConf already exist. path: "  + queryConfPath.toString());
+      }
+
+      LocalResource queryConfSrc = createApplicationResource(fsCtx,
+          queryConfPath, LocalResourceType.FILE);
+//        localResources.put(QueryConf.FILENAME,  queryConfSrc);
+      localResources.put(queryConfPath.getName(), queryConfSrc);
+
+      ctx.setLocalResources(localResources);
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    // TODO - move to sub-class
+    // Add shuffle token
+    Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+    try {
+      //LOG.info("Putting shuffle token in serviceData");
+      serviceData.put(PullServerAuxService.PULLSERVER_SERVICEID,
+          PullServerAuxService.serializeMetaData(0));
+    } catch (IOException ioe) {
+      LOG.error(ioe);
+    }
+    ctx.setServiceData(serviceData);
+
+    return ctx;
+  }
+
+  private static LocalResource createApplicationResource(FileContext fs,
+                                                         Path p, LocalResourceType type)
+      throws IOException {
+    LocalResource rsrc = recordFactory.newRecordInstance(LocalResource.class);
+    FileStatus rsrcStat = fs.getFileStatus(p);
+    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs
+        .getDefaultFileSystem().resolvePath(rsrcStat.getPath())));
+    rsrc.setSize(rsrcStat.getLen());
+    rsrc.setTimestamp(rsrcStat.getModificationTime());
+    rsrc.setType(type);
+    rsrc.setVisibility(LocalResourceVisibility.APPLICATION);
+    return rsrc;
+  }
+
+  private static void writeConf(Configuration conf, Path queryConfFile)
+      throws IOException {
+    // Write job file to Tajo's fs
+    FileSystem fs = queryConfFile.getFileSystem(conf);
+    FSDataOutputStream out =
+        FileSystem.create(fs, queryConfFile,
+            new FsPermission(QUERYCONF_FILE_PERMISSION));
+    try {
+      conf.writeXml(out);
+    } finally {
+      out.close();
+    }
+  }
+
+  @Override
+  public synchronized void stopContainer() {
+
+    if(isCompletelyDone()) {
+      return;
+    }
+    if(this.state == ContainerState.PREP) {
+      this.state = ContainerState.KILLED_BEFORE_LAUNCH;
+    } else {
+      LOG.info("KILLING " + containerID);
+
+      ContainerManager proxy = null;
+      try {
+        proxy = getCMProxy(this.containerID, this.containerMgrAddress,
+            this.containerToken);
+
+        // kill the remote container if already launched
+        StopContainerRequest stopRequest = Records
+            .newRecord(StopContainerRequest.class);
+        stopRequest.setContainerId(this.containerID);
+        proxy.stopContainer(stopRequest);
+        // If stopContainer returns without an error, assuming the stop made
+        // it over to the NodeManager.
+//          context.getEventHandler().handle(
+//              new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
+        context.getResourceAllocator().removeContainer(containerID);
+      } catch (Throwable t) {
+
+        // ignore the cleanup failure
+        String message = "cleanup failed for container "
+            + this.containerID + " : "
+            + StringUtils.stringifyException(t);
+//          context.getEventHandler().handle(
+//              new AMContainerEventStopFailed(containerID, message));
+        LOG.warn(message);
+        this.state = ContainerState.DONE;
+        return;
+      } finally {
+        if (proxy != null) {
+          yarnRPC.stopProxy(proxy, conf);
+        }
+      }
+      this.state = ContainerState.DONE;
+    }
+  }
+
+  protected Vector<CharSequence> getTaskParams() {
+    String queryMasterHost = context.getQueryMasterContext().getWorkerContext()
+        .getTajoWorkerManagerService().getBindAddr().getHostName();
+    int queryMasterPort = context.getQueryMasterContext().getWorkerContext()
+        .getTajoWorkerManagerService().getBindAddr().getPort();
+
+    Vector<CharSequence> taskParams = new Vector<CharSequence>();
+    taskParams.add(queryMasterHost); // queryMaster hostname
+    taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
+    taskParams.add(context.getOutputPath().toString());
+    return taskParams;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
new file mode 100644
index 0000000..5ac4fb5
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class YarnTaskRunnerLauncherImpl extends AbstractService implements TaskRunnerLauncher {
+
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(YarnTaskRunnerLauncherImpl.class);
+  //private final YarnRPC yarnRPC;
+  private final static RecordFactory recordFactory =
+      RecordFactoryProvider.getRecordFactory(null);
+  private QueryMasterTask.QueryContext context;
+
+  // For ContainerLauncherSpec
+  private static AtomicBoolean initialClasspathFlag = new AtomicBoolean();
+  private static String initialClasspath = null;
+  private static final Object classpathLock = new Object();
+  private ContainerLaunchContext commonContainerSpec = null;
+
+  final public static FsPermission QUERYCONF_FILE_PERMISSION =
+      FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+  /** for launching TaskRunners in parallel */
+  private final ExecutorService executorService;
+
+  private YarnRPC yarnRPC;
+
+  public YarnTaskRunnerLauncherImpl(QueryMasterTask.QueryContext context, YarnRPC yarnRPC) {
+    super(YarnTaskRunnerLauncherImpl.class.getName());
+    this.context = context;
+    this.yarnRPC = yarnRPC;
+    executorService = Executors.newFixedThreadPool(
+        context.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+  }
+
+  public void start() {
+    super.start();
+  }
+
+  public void stop() {
+    executorService.shutdownNow();
+
+    while(!executorService.isTerminated()) {
+      LOG.info("====>executorService.isTerminated:" + executorService.isTerminated() + "," + executorService.isShutdown());
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+      }
+    }
+    Map<ContainerId, ContainerProxy> containers = context.getResourceAllocator().getContainers();
+    for(ContainerProxy eachProxy: containers.values()) {
+      try {
+        eachProxy.stopContainer();
+      } catch (Exception e) {
+      }
+    }
+    super.stop();
+  }
+
+  @Override
+  public void handle(TaskRunnerGroupEvent event) {
+    if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) {
+     launchTaskRunners(event.executionBlockId, event.getContainers());
+    } else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
+      stopTaskRunners(event.getContainers());
+    }
+  }
+
+  private void launchTaskRunners(ExecutionBlockId executionBlockId, Collection<Container> containers) {
+    commonContainerSpec = YarnContainerProxy.createCommonContainerLaunchContext(getConfig(),
+        executionBlockId.getQueryId().toString(), false);
+    for (Container container : containers) {
+      final ContainerProxy proxy = new YarnContainerProxy(context, getConfig(),
+          yarnRPC, container, executionBlockId);
+      executorService.submit(new LaunchRunner(container.getId(), proxy));
+    }
+  }
+
+  protected class LaunchRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+      this.proxy = proxy;
+      this.id = id;
+    }
+    @Override
+    public void run() {
+      proxy.launch(commonContainerSpec);
+      LOG.info("ContainerProxy started:" + id);
+    }
+  }
+
+  private void stopTaskRunners(Collection<Container> containers) {
+    for (Container container : containers) {
+      final ContainerProxy proxy = context.getResourceAllocator().getContainer(container.getId());
+      executorService.submit(new StopContainerRunner(container.getId(), proxy));
+    }
+  }
+
+  private class StopContainerRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+      this.id = id;
+      this.proxy = proxy;
+    }
+
+    @Override
+    public void run() {
+      proxy.stopContainer();
+      LOG.info("ContainerProxy stopped:" + id);
+    }
+  }
+
+
+  /**
+   * Lock this on initialClasspath so that there is only one fork in the AM for
+   * getting the initial class-path. TODO: We already construct
+   * a parent CLC and use it for all the containers, so this should go away
+   * once the mr-generated-classpath stuff is gone.
+   */
+  private static String getInitialClasspath(Configuration conf) {
+    synchronized (classpathLock) {
+      if (initialClasspathFlag.get()) {
+        return initialClasspath;
+      }
+      Map<String, String> env = new HashMap<String, String>();
+
+      initialClasspath = env.get(Environment.CLASSPATH.name());
+      initialClasspathFlag.set(true);
+      return initialClasspath;
+    }
+  }
+
+//  public class TaskRunnerContainerProxy extends ContainerProxy {
+//    private final ExecutionBlockId executionBlockId;
+//
+//    public TaskRunnerContainerProxy(QueryMasterTask.QueryContext context, Configuration conf, YarnRPC yarnRPC,
+//                                    Container container, ExecutionBlockId executionBlockId) {
+//      super(context, conf, yarnRPC, container);
+//      this.executionBlockId = executionBlockId;
+//    }
+//
+//    @Override
+//    protected void containerStarted() {
+//      context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+//    }
+//
+//    @Override
+//    protected String getId() {
+//      return executionBlockId.toString();
+//    }
+//
+//    @Override
+//    protected String getRunnerClass() {
+//      return TaskRunner.class.getCanonicalName();
+//    }
+//
+//    @Override
+//    protected Vector<CharSequence> getTaskParams() {
+//      Vector<CharSequence> taskParams = new Vector<CharSequence>();
+//      taskParams.add(queryMasterHost); // queryMaster hostname
+//      taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
+//
+//      return taskParams;
+//    }
+//  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
index 6704aa4..ee594a3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
@@ -21,11 +21,11 @@ package org.apache.tajo.master.event;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 
 public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEventType>  {
 
-  private final SubQueryId subQueryId;
+  private final ExecutionBlockId executionBlockId;
   private final Priority priority;
   private final Resource resource;
   private final boolean isLeafQuery;
@@ -33,13 +33,13 @@ public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEv
   private final float progress;
 
   public ContainerAllocationEvent(ContainerAllocatorEventType eventType,
-                                         SubQueryId subQueryId,
-                                         Priority priority,
-                                         Resource resource,
-                                         int requiredNum,
-                                         boolean isLeafQuery, float progress) {
+                                  ExecutionBlockId executionBlockId,
+                                  Priority priority,
+                                  Resource resource,
+                                  int requiredNum,
+                                  boolean isLeafQuery, float progress) {
     super(eventType);
-    this.subQueryId = subQueryId;
+    this.executionBlockId = executionBlockId;
     this.priority = priority;
     this.resource = resource;
     this.requiredNum = requiredNum;
@@ -47,8 +47,8 @@ public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEv
     this.progress = progress;
   }
 
-  public SubQueryId getSubQueryId() {
-    return subQueryId;
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
   }
 
   public Priority getPriority() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
index 44abf30..c34b174 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
@@ -20,7 +20,7 @@ package org.apache.tajo.master.event;
 
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 
 import java.util.Map;
 
@@ -29,12 +29,12 @@ public class GrouppedContainerAllocatorEvent
   private final Map<String, Integer> requestMap;
 
   public GrouppedContainerAllocatorEvent(ContainerAllocatorEventType eventType,
-                                         SubQueryId subQueryId,
+                                         ExecutionBlockId executionBlockId,
                                          Priority priority,
                                          Resource resource,
                                          Map<String, Integer> requestMap,
                                          boolean isLeafQuery, float progress) {
-    super(eventType, subQueryId, priority,
+    super(eventType, executionBlockId, priority,
         resource, requestMap.size(), isLeafQuery, progress);
     this.requestMap = requestMap;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
new file mode 100644
index 0000000..6ae8ff7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryId;
+
+public class QueryStartEvent extends AbstractEvent {
+  public enum EventType {
+    QUERY_START
+  }
+
+  private QueryId queryId;
+  private String logicalPlanJson;
+
+  public QueryStartEvent(QueryId queryId, String logicalPlanJson) {
+    super(EventType.QUERY_START);
+    this.queryId = queryId;
+    this.logicalPlanJson = logicalPlanJson;
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public String getLogicalPlanJson() {
+    return logicalPlanJson;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getName() + "," + getType() + "," + queryId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
index 10b67fe..ae36a69 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
@@ -18,18 +18,18 @@
 
 package org.apache.tajo.master.event;
 
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 
 public class QuerySubQueryEvent extends QueryEvent {
-  private SubQueryId subQueryId;
+  private ExecutionBlockId executionBlockId;
 
-  public QuerySubQueryEvent(final SubQueryId id,
+  public QuerySubQueryEvent(final ExecutionBlockId id,
                             final QueryEventType queryEvent) {
     super(id.getQueryId(), queryEvent);
-    this.subQueryId = id;
+    this.executionBlockId = id;
   }
 
-  public SubQueryId getSubQueryId() {
-    return this.subQueryId;
+  public ExecutionBlockId getExecutionBlockId() {
+    return this.executionBlockId;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
index 26c7231..7e07525 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
@@ -18,22 +18,22 @@
 
 package org.apache.tajo.master.event;
 
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.master.querymaster.SubQueryState;
 
 public class SubQueryCompletedEvent extends QueryEvent {
-  private final SubQueryId subQueryId;
+  private final ExecutionBlockId executionBlockId;
   private final SubQueryState finalState;
 
-  public SubQueryCompletedEvent(final SubQueryId subQueryId,
+  public SubQueryCompletedEvent(final ExecutionBlockId executionBlockId,
                                 SubQueryState finalState) {
-    super(subQueryId.getQueryId(), QueryEventType.SUBQUERY_COMPLETED);
-    this.subQueryId = subQueryId;
+    super(executionBlockId.getQueryId(), QueryEventType.SUBQUERY_COMPLETED);
+    this.executionBlockId = executionBlockId;
     this.finalState = finalState;
   }
 
-  public SubQueryId getSubQueryId() {
-    return subQueryId;
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
   }
 
   public SubQueryState getFinalState() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
index 5c0ef9a..a8f4800 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
@@ -19,14 +19,14 @@
 package org.apache.tajo.master.event;
 
 import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 
 import java.util.List;
 
 public class SubQueryContainerAllocationEvent extends SubQueryEvent {
   private List<Container> allocatedContainer;
 
-  public SubQueryContainerAllocationEvent(final SubQueryId id,
+  public SubQueryContainerAllocationEvent(final ExecutionBlockId id,
                                           List<Container> allocatedContainer) {
     super(id, SubQueryEventType.SQ_CONTAINER_ALLOCATED);
     this.allocatedContainer = allocatedContainer;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
index 11470ed..2b3d598 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
@@ -19,17 +19,17 @@
 package org.apache.tajo.master.event;
 
 import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 
 public class SubQueryEvent extends AbstractEvent<SubQueryEventType> {
-  private final SubQueryId id;
+  private final ExecutionBlockId id;
 
-  public SubQueryEvent(SubQueryId id, SubQueryEventType subQueryEventType) {
+  public SubQueryEvent(ExecutionBlockId id, SubQueryEventType subQueryEventType) {
     super(subQueryEventType);
     this.id = id;
   }
 
-  public SubQueryId getSubQueryId() {
+  public ExecutionBlockId getSubQueryId() {
     return id;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
index d85d4f2..e02196a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
@@ -18,14 +18,14 @@
 
 package org.apache.tajo.master.event;
 
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.master.querymaster.SubQueryState;
 
 public class SubQuerySucceeEvent extends SubQueryCompletedEvent {
   private final TableMeta tableMeta;
 
-  public SubQuerySucceeEvent(final SubQueryId id, TableMeta tableMeta) {
+  public SubQuerySucceeEvent(final ExecutionBlockId id, TableMeta tableMeta) {
     super(id, SubQueryState.SUCCEEDED);
     this.tableMeta = tableMeta;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
index 0315236..0217f20 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
@@ -27,7 +27,7 @@ public class SubQueryTaskEvent extends SubQueryEvent {
   private QueryUnitId taskId;
   public SubQueryTaskEvent(QueryUnitId taskId,
                            SubQueryEventType subQueryEventType) {
-    super(taskId.getSubQueryId(), subQueryEventType);
+    super(taskId.getExecutionBlockId(), subQueryEventType);
     this.taskId = taskId;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
index bc84011..d980e05 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.master.event;
 
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskStatusProto;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskStatusProto;
 
 public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent {
   private final TaskStatusProto status;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
index e3a4b5f..3ee389a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.master.event;
 
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskCompletionReport;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
 
 public class TaskCompletionEvent extends TaskAttemptEvent {
   private TaskCompletionReport report;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
index 06fb392..d70de8a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.master.event;
 
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskFatalErrorReport;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskFatalErrorReport;
 
 public class TaskFatalErrorEvent extends TaskAttemptEvent {
   private TaskFatalErrorReport report;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
index 166e103..9be7cab 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
@@ -21,7 +21,8 @@ package org.apache.tajo.master.event;
 import com.google.protobuf.RpcCallback;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryUnitRequestProto;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
 import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
 
 public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
@@ -31,12 +32,16 @@ public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
   }
 
   private final ContainerId workerId;
+  private final ExecutionBlockId executionBlockId;
+
   private final RpcCallback<QueryUnitRequestProto> callback;
 
   public TaskRequestEvent(ContainerId workerId,
+                          ExecutionBlockId executionBlockId,
                           RpcCallback<QueryUnitRequestProto> callback) {
     super(TaskRequestEventType.TASK_REQ);
     this.workerId = workerId;
+    this.executionBlockId = executionBlockId;
     this.callback = callback;
   }
 
@@ -44,6 +49,10 @@ public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
     return this.workerId;
   }
 
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
+  }
+
   public RpcCallback<QueryUnitRequestProto> getCallback() {
     return this.callback;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
index 1f87356..f460203 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
@@ -34,7 +34,7 @@ public class TaskScheduleEvent extends TaskSchedulerEvent {
                            final EventType eventType, boolean isLeafQuery,
                            final List<QueryUnit.DataLocation> dataLocations,
                            final String[] racks) {
-    super(eventType, attemptId.getSubQueryId());
+    super(eventType, attemptId.getQueryUnitId().getExecutionBlockId());
     this.attemptId = attemptId;
     this.isLeafQuery = isLeafQuery;
     this.dataLocations = dataLocations;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
index d73bb87..71d8587 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tajo.master.event;
 
 import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
 
 public class TaskSchedulerEvent extends AbstractEvent<EventType> {
@@ -28,14 +28,14 @@ public class TaskSchedulerEvent extends AbstractEvent<EventType> {
     T_SUBQUERY_COMPLETED
   }
 
-  private final SubQueryId subQueryId;
+  private final ExecutionBlockId executionBlockId;
 
-  public TaskSchedulerEvent(EventType eventType, SubQueryId subQueryId) {
+  public TaskSchedulerEvent(EventType eventType, ExecutionBlockId queryBlockId) {
     super(eventType);
-    this.subQueryId = subQueryId;
+    this.executionBlockId = queryBlockId;
   }
 
-  public SubQueryId getSubQueryId() {
-    return this.subQueryId;
+  public ExecutionBlockId getExecutionBlockId() {
+    return this.executionBlockId;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 3179abf..99b7c62 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryConf;
 import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableDescImpl;
@@ -35,7 +35,6 @@ import org.apache.tajo.engine.planner.global.MasterPlan;
 import org.apache.tajo.master.ExecutionBlock;
 import org.apache.tajo.master.ExecutionBlockCursor;
 import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
 import org.apache.tajo.storage.StorageManager;
 
 import java.io.IOException;
@@ -50,16 +49,15 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 public class Query implements EventHandler<QueryEvent> {
   private static final Log LOG = LogFactory.getLog(Query.class);
 
-
   // Facilities for Query
   private final QueryConf conf;
   private final Clock clock;
   private String queryStr;
-  private Map<SubQueryId, SubQuery> subqueries;
+  private Map<ExecutionBlockId, SubQuery> subqueries;
   private final EventHandler eventHandler;
   private final MasterPlan plan;
   private final StorageManager sm;
-  private QueryContext context;
+  private QueryMasterTask.QueryContext context;
   private ExecutionBlockCursor cursor;
 
   // Query Status
@@ -106,22 +104,21 @@ public class Query implements EventHandler<QueryEvent> {
 
       .installTopology();
 
-  public Query(final QueryContext context, final QueryId id, Clock clock,
+  public Query(final QueryMasterTask.QueryContext context, final QueryId id,
                final long appSubmitTime,
                final String queryStr,
                final EventHandler eventHandler,
-               final MasterPlan plan,
-               final StorageManager sm) {
+               final MasterPlan plan) {
     this.context = context;
     this.conf = context.getConf();
     this.id = id;
-    this.clock = clock;
+    this.clock = context.getClock();
     this.appSubmitTime = appSubmitTime;
     this.queryStr = queryStr;
     subqueries = Maps.newHashMap();
     this.eventHandler = eventHandler;
     this.plan = plan;
-    this.sm = sm;
+    this.sm = context.getStorageManager();
     cursor = new ExecutionBlockCursor(plan);
 
     ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
@@ -131,23 +128,19 @@ public class Query implements EventHandler<QueryEvent> {
     stateMachine = stateMachineFactory.make(this);
   }
 
-  public boolean isCreateTableStmt() {
-    return context.isCreateTableQuery();
-  }
-
-//  protected FileSystem getFileSystem(Configuration conf) throws IOException {
-//    return FileSystem.get(conf);
-//  }
-
   public float getProgress() {
     QueryState state = getStateMachine().getCurrentState();
     if (state == QueryState.QUERY_SUCCEEDED) {
       return 1.0f;
     } else {
       int idx = 0;
-      float [] subProgresses = new float[subqueries.size()];
+      List<SubQuery> tempSubQueries = new ArrayList<SubQuery>();
+      synchronized(subqueries) {
+        tempSubQueries.addAll(subqueries.values());
+      }
+      float [] subProgresses = new float[tempSubQueries.size()];
       boolean finished = true;
-      for (SubQuery subquery: subqueries.values()) {
+      for (SubQuery subquery: tempSubQueries) {
         if (subquery.getState() != SubQueryState.NEW) {
           subProgresses[idx] = subquery.getProgress();
           if (finished && subquery.getState() != SubQueryState.SUCCEEDED) {
@@ -239,8 +232,8 @@ public class Query implements EventHandler<QueryEvent> {
   public QueryId getId() {
     return this.id;
   }
-  
-  public SubQuery getSubQuery(SubQueryId id) {
+
+  public SubQuery getSubQuery(ExecutionBlockId id) {
     return this.subqueries.get(id);
   }
 
@@ -263,7 +256,7 @@ public class Query implements EventHandler<QueryEvent> {
     @Override
     public QueryState transition(Query query, QueryEvent queryEvent) {
       query.setStartTime();
-      query.context.setState(QueryState.QUERY_INIT);
+      //query.context.setState(QueryState.QUERY_INIT);
       return QueryState.QUERY_INIT;
     }
   }
@@ -277,7 +270,8 @@ public class Query implements EventHandler<QueryEvent> {
           query.sm);
       subQuery.setPriority(query.priority--);
       query.addSubQuery(subQuery);
-      LOG.info("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+      LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+
       subQuery.handle(new SubQueryEvent(subQuery.getId(),
           SubQueryEventType.SQ_INIT));
     }
@@ -301,13 +295,16 @@ public class Query implements EventHandler<QueryEvent> {
           query.addSubQuery(nextSubQuery);
           nextSubQuery.handle(new SubQueryEvent(nextSubQuery.getId(),
               SubQueryEventType.SQ_INIT));
-          LOG.info("Scheduling SubQuery's Priority: " + nextSubQuery.getPriority());
-          LOG.info("Scheduling SubQuery's Plan: \n" + nextSubQuery.getBlock().getPlan());
+          LOG.info("Scheduling SubQuery:" + nextSubQuery.getId());
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Scheduling SubQuery's Priority: " + nextSubQuery.getPriority());
+            LOG.debug("Scheduling SubQuery's Plan: \n" + nextSubQuery.getBlock().getPlan());
+          }
           return query.checkQueryForCompleted();
 
         } else { // Finish a query
           if (query.checkQueryForCompleted() == QueryState.QUERY_SUCCEEDED) {
-            SubQuery subQuery = query.getSubQuery(castEvent.getSubQueryId());
+            SubQuery subQuery = query.getSubQuery(castEvent.getExecutionBlockId());
             TableDesc desc = new TableDescImpl(query.conf.getOutputTable(),
                 subQuery.getTableMeta(), query.context.getOutputPath());
             query.setResultDesc(desc);
@@ -319,7 +316,7 @@ public class Query implements EventHandler<QueryEvent> {
             query.eventHandler.handle(new QueryFinishEvent(query.getId()));
 
             if (query.context.isCreateTableQuery()) {
-              // TOOD move to QueryMasterManager
+              // TOOD move to QueryJobManager
               //query.context.getCatalog().addTable(desc);
             }
           }
@@ -363,7 +360,6 @@ public class Query implements EventHandler<QueryEvent> {
 
   public QueryState finished(QueryState finalState) {
     setFinishTime();
-    context.setState(finalState);
     return finalState;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
new file mode 100644
index 0000000..2e2870f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.master.rm.WorkerResourceManager;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.ProtoAsyncRpcClient;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class QueryInProgress extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(QueryInProgress.class.getName());
+
+  private QueryId queryId;
+
+  private AsyncDispatcher dispatcher;
+
+  private LogicalRootNode plan;
+
+  private AtomicBoolean querySubmitted = new AtomicBoolean(false);
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private QueryInfo queryInfo;
+
+  private final TajoMaster.MasterContext masterContext;
+
+  private ProtoAsyncRpcClient queryMasterRpc;
+
+  private TajoWorkerProtocol.TajoWorkerProtocolService queryMasterRpcClient;
+
+  public QueryInProgress(
+      TajoMaster.MasterContext masterContext,
+      QueryId queryId, String sql, LogicalRootNode plan) {
+    super(QueryInProgress.class.getName());
+    this.masterContext = masterContext;
+    this.queryId = queryId;
+    this.plan = plan;
+
+    queryInfo = new QueryInfo(queryId, sql);
+    queryInfo.setStartTime(System.currentTimeMillis());
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    dispatcher = new AsyncDispatcher();
+    this.addService(dispatcher);
+
+    dispatcher.register(QueryJobEvent.Type.class, new QueryInProgressEventHandler());
+    super.init(conf);
+  }
+
+  @Override
+  public void stop() {
+    synchronized(stopped) {
+      if(stopped.get()) {
+        return;
+      }
+      stopped.set(true);
+    }
+    LOG.info("=========================================================");
+    LOG.info("Stop query:" + queryId);
+
+    masterContext.getResourceManager().stopQueryMaster(queryId);
+
+    boolean queryMasterStopped = false;
+    long startTime = System.currentTimeMillis();
+    while(true) {
+      try {
+        if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
+          LOG.info("====> " + queryId + " QueryMaster stopped");
+          queryMasterStopped = true;
+          break;
+        }
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      }
+
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        break;
+      }
+      if(System.currentTimeMillis() - startTime > 60 * 1000) {
+        LOG.warn("Failed to stop QueryMaster:" + queryId);
+        break;
+      }
+    }
+
+    if(queryMasterRpc != null) {
+      //TODO release to connection pool
+      queryMasterRpc.close();
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  public EventHandler getEventHandler() {
+    return dispatcher.getEventHandler();
+  }
+
+  public void startQueryMaster() {
+    try {
+      LOG.info("Initializing QueryInProgress for QueryID=" + queryId);
+      WorkerResourceManager resourceManager = masterContext.getResourceManager();
+      WorkerResource queryMasterResource = resourceManager.allocateQueryMaster(this);
+
+      if(queryMasterResource != null) {
+        queryInfo.setQueryMasterResource(queryMasterResource);
+      }
+      getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInfo));
+    } catch (Exception e) {
+      catchException(e);
+    }
+  }
+
+  class QueryInProgressEventHandler implements EventHandler<QueryJobEvent> {
+    @Override
+    public void handle(QueryJobEvent queryJobEvent) {
+      if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_HEARTBEAT) {
+        heartbeat(queryJobEvent.getQueryInfo());
+      } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_MASTER_START) {
+        masterContext.getResourceManager().startQueryMaster(QueryInProgress.this);
+      } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_START) {
+        submmitQueryToMaster();
+      } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_FINISH) {
+        stop();
+      }
+    }
+  }
+
+  public TajoWorkerProtocol.TajoWorkerProtocolService getQueryMasterRpcClient() {
+    return queryMasterRpcClient;
+  }
+
+  private void connectQueryMaster() throws Exception {
+    if(queryInfo.getQueryMasterResource() != null &&
+        queryInfo.getQueryMasterResource().getAllocatedHost() != null) {
+      InetSocketAddress addr = NetUtils.createSocketAddr(
+          queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
+      LOG.info("Connect to QueryMaster:" + addr);
+      //TODO Get Connection from pool
+      queryMasterRpc = new ProtoAsyncRpcClient(TajoWorkerProtocol.class, addr);
+      queryMasterRpcClient = queryMasterRpc.getStub();
+    }
+  }
+
+  private synchronized void submmitQueryToMaster() {
+    if(querySubmitted.get()) {
+      return;
+    }
+
+    try {
+      if(queryMasterRpcClient == null) {
+        connectQueryMaster();
+      }
+      if(queryMasterRpcClient == null) {
+        LOG.info("No QueryMaster conneciton info.");
+        //TODO wait
+        return;
+      }
+      LOG.info("====>Call executeQuery to :" +
+          queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId);
+      queryMasterRpcClient.executeQuery(
+          null,
+          TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder()
+              .setQueryId(queryId.getProto())
+              .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build())
+              .build(), NullCallback.get());
+      querySubmitted.set(true);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  public void catchException(Exception e) {
+    LOG.error(e.getMessage(), e);
+    queryInfo.setQueryState(TajoProtos.QueryState.QUERY_FAILED);
+    queryInfo.setLastMessage(StringUtils.stringifyException(e));
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public QueryInfo getQueryInfo() {
+    return this.queryInfo;
+  }
+
+  private void heartbeat(QueryInfo queryInfo) {
+    LOG.info("Received QueryMaster heartbeat:" + queryInfo);
+    if(queryInfo.getQueryMasterResource() != null) {
+      this.queryInfo.setQueryMasterResource(queryInfo.getQueryMasterResource());
+    }
+    this.queryInfo.setQueryState(queryInfo.getQueryState());
+
+    if(queryInfo.getLastMessage() != null && !queryInfo.getLastMessage().isEmpty()) {
+      this.queryInfo.setLastMessage(queryInfo.getLastMessage());
+      LOG.info(queryId + queryInfo.getLastMessage());
+    }
+    if(this.queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_FAILED) {
+      //TODO needed QueryMaster's detail status(failed before or after launching worker)
+      //queryMasterStopped.set(true);
+      LOG.warn(queryId + " failed, " + queryInfo.getLastMessage());
+    }
+
+    if(!querySubmitted.get()) {
+      getEventHandler().handle(
+          new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, this.queryInfo));
+    }
+
+    if(isFinishState(this.queryInfo.getQueryState())) {
+      getEventHandler().handle(
+          new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_FINISH, this.queryInfo));
+    }
+  }
+
+  private boolean isFinishState(TajoProtos.QueryState state) {
+    return state == TajoProtos.QueryState.QUERY_FAILED ||
+        state == TajoProtos.QueryState.QUERY_KILLED ||
+        state == TajoProtos.QueryState.QUERY_SUCCEEDED;
+  }
+
+//  private void checkQueryMasterShutdown() {
+//    //run background
+//    Thread t = new Thread() {
+//      public void run() {
+//        while(true) {
+//          try {
+//            if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
+//              queryMasterStopped.set(true);
+//              LOG.info("==========> " + queryId + " QueryMaster stopped");
+//              break;
+//            }
+//          } catch (Exception e) {
+//            LOG.error(e.getMessage(), e);
+//          }
+//          try {
+//            Thread.sleep(1000);
+//          } catch (InterruptedException e) {
+//            break;
+//          }
+//        }
+//      }
+//    };
+//
+//    t.start();
+//  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
new file mode 100644
index 0000000..e7ceae7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.master.rm.WorkerResource;
+
+public class QueryInfo {
+  private QueryId queryId;
+  private String sql;
+  private TajoProtos.QueryState queryState;
+  private float progress;
+  private long startTime;
+  private long finishTime;
+  private String lastMessage;
+  private WorkerResource queryMasterResource;
+
+  public QueryInfo(QueryId queryId) {
+    this(queryId, null);
+  }
+
+  public QueryInfo(QueryId queryId, String sql) {
+    this.queryId = queryId;
+    this.sql = sql;
+    this.queryState = TajoProtos.QueryState.QUERY_MASTER_INIT;
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public String getSql() {
+    return sql;
+  }
+
+  public String getQueryMasterHost() {
+    if(queryMasterResource == null) {
+      return null;
+    }
+    return queryMasterResource.getAllocatedHost();
+  }
+
+  public void setQueryMasterResource(WorkerResource queryMasterResource) {
+    this.queryMasterResource = queryMasterResource;
+  }
+
+  public int getQueryMasterPort() {
+    if(queryMasterResource == null) {
+      return 0;
+    }
+    return queryMasterResource.getPorts()[0];
+  }
+
+  public int getQueryMasterClientPort() {
+    if(queryMasterResource == null) {
+      return 0;
+    }
+    return queryMasterResource.getPorts()[1];
+  }
+
+  public TajoProtos.QueryState getQueryState() {
+    return queryState;
+  }
+
+  public void setQueryState(TajoProtos.QueryState queryState) {
+    this.queryState = queryState;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public String getLastMessage() {
+    return lastMessage;
+  }
+
+  public void setLastMessage(String lastMessage) {
+    this.lastMessage = lastMessage;
+  }
+
+  public WorkerResource getQueryMasterResource() {
+    return queryMasterResource;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public void setProgress(float progress) {
+    this.progress = progress;
+  }
+
+  @Override
+  public String toString() {
+    return queryId.toString() + ", queryMaster=" + queryMasterResource;
+  }
+}


[3/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
new file mode 100644
index 0000000..8756eed
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
@@ -0,0 +1,337 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.client.YarnClient;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.YarnContainerProxy;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryJobEvent;
+import org.apache.tajo.util.ApplicationIdUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.util.*;
+
+public class YarnTajoResourceManager implements WorkerResourceManager {
+  private static final Log LOG = LogFactory.getLog(YarnTajoResourceManager.class);
+
+  private YarnClient yarnClient;
+  private AMRMProtocol rmClient;
+  private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+  private Configuration conf;
+  private TajoMaster.MasterContext masterContext;
+
+  public YarnTajoResourceManager() {
+
+  }
+
+  public YarnTajoResourceManager(TajoMaster.MasterContext masterContext) {
+    this.masterContext = masterContext;
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  @Override
+  public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request) {
+    //nothing to do
+    //yarn manages worker membership.
+  }
+
+  @Override
+  public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource) {
+    //nothing to do
+  }
+
+  @Override
+  public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress) {
+    //nothing to do
+    //allocateAndLaunchQueryMaster in startQueryMaster()
+    return null;
+  }
+
+  @Override
+  public void allocateWorkerResources(
+      TajoMasterProtocol.WorkerResourceAllocationRequest request,
+      RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> rpcCallBack) {
+    //nothing to do
+  }
+
+  @Override
+  public void startQueryMaster(QueryInProgress queryInProgress) {
+    try {
+      allocateAndLaunchQueryMaster(queryInProgress);
+
+      queryInProgress.getEventHandler().handle(
+          new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
+
+    } catch (YarnRemoteException e) {
+      LOG.error(e.getMessage(), e);
+      //TODO set QueryState(fail)
+    }
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.conf = conf;
+    connectYarnClient();
+
+    final YarnConfiguration yarnConf = new YarnConfiguration(conf);
+    final YarnRPC rpc = YarnRPC.create(conf);
+    final InetSocketAddress rmAddress = conf.getSocketAddr(
+        YarnConfiguration.RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
+
+    UserGroupInformation currentUser;
+    try {
+      currentUser = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new YarnException(e);
+    }
+
+    rmClient = currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
+      @Override
+      public AMRMProtocol run() {
+        return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rmAddress, yarnConf);
+      }
+    });
+  }
+
+  @Override
+  public String getSeedQueryId() throws IOException {
+    try {
+      GetNewApplicationResponse newApp = yarnClient.getNewApplication();
+      ApplicationId appId = newApp.getApplicationId();
+
+      return appId.toString();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+
+      throw new IOException(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void stopQueryMaster(QueryId queryId) {
+    try {
+      FinalApplicationStatus appStatus = FinalApplicationStatus.UNDEFINED;
+      QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+      if(queryInProgress != null) {
+        return;
+      }
+      TajoProtos.QueryState state = queryInProgress.getQueryInfo().getQueryState();
+      if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+        appStatus = FinalApplicationStatus.SUCCEEDED;
+      } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
+        appStatus = FinalApplicationStatus.FAILED;
+      } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
+        appStatus = FinalApplicationStatus.FAILED;
+      }
+      FinishApplicationMasterRequest request = recordFactory
+          .newRecordInstance(FinishApplicationMasterRequest.class);
+      request.setAppAttemptId(ApplicationIdUtils.createApplicationAttemptId(queryId));
+      request.setFinishApplicationStatus(appStatus);
+      request.setDiagnostics("QueryMaster shutdown by TajoMaster.");
+      rmClient.finishApplicationMaster(request);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  private void connectYarnClient() {
+    this.yarnClient = new YarnClientImpl();
+    this.yarnClient.init(conf);
+    this.yarnClient.start();
+  }
+
+  private ApplicationAttemptId allocateAndLaunchQueryMaster(QueryInProgress queryInProgress)
+      throws YarnRemoteException {
+    QueryId queryId = queryInProgress.getQueryId();
+    ApplicationId appId = ApplicationIdUtils.queryIdToAppId(queryId);
+
+    LOG.info("Allocate and launch ApplicationMaster for QueryMaster: queryId=" +
+        queryId + ", appId=" + appId);
+
+    ApplicationSubmissionContext appContext = Records.newRecord(ApplicationSubmissionContext.class);
+
+    // set the application id
+    appContext.setApplicationId(appId);
+    // set the application name
+    appContext.setApplicationName("Tajo");
+
+    Priority pri = Records.newRecord(Priority.class);
+    pri.setPriority(5);
+    appContext.setPriority(pri);
+
+    // Set the queue to which this application is to be submitted in the RM
+    appContext.setQueue("default");
+
+    ContainerLaunchContext commonContainerLaunchContext =
+        YarnContainerProxy.createCommonContainerLaunchContext(masterContext.getConf(), queryId.toString(), true);
+
+    // Setup environment by cloning from common env.
+    Map<String, String> env = commonContainerLaunchContext.getEnvironment();
+    Map<String, String> myEnv = new HashMap<String, String>(env.size());
+    myEnv.putAll(env);
+
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the local resources
+    ////////////////////////////////////////////////////////////////////////////
+    // Set the necessary command to execute the application master
+    Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+    // Set java executable command
+    //LOG.info("Setting up app master command");
+    vargs.add("${JAVA_HOME}" + "/bin/java");
+    // Set Xmx based on am memory size
+    String jvmOptions = masterContext.getConf().get("tajo.rm.yarn.querymaster.jvm.option", "-Xmx2000m");
+
+    for(String eachToken: jvmOptions.split((" "))) {
+      vargs.add(eachToken);
+    }
+    // Set Remote Debugging
+    //if (!context.getQuery().getSubQuery(event.getExecutionBlockId()).isLeafQuery()) {
+    //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
+    //}
+    // Set class name
+    //vargs.add(QueryMasterRunner.class.getCanonicalName());
+    vargs.add(TajoWorker.class.getCanonicalName());
+    vargs.add("qm");
+    vargs.add(queryId.toString()); // queryId
+    vargs.add(masterContext.getTajoMasterService().getBindAddress().getHostName() + ":" +
+        masterContext.getTajoMasterService().getBindAddress().getPort());
+
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+    // Get final commmand
+    StringBuilder command = new StringBuilder();
+    for (CharSequence str : vargs) {
+      command.append(str).append(" ");
+    }
+
+    LOG.info("Completed setting up QueryMasterRunner command " + command.toString());
+    List<String> commands = new ArrayList<String>();
+    commands.add(command.toString());
+
+    final Resource resource = Records.newRecord(Resource.class);
+    // TODO - get default value from conf
+    resource.setMemory(256);
+    resource.setVirtualCores(1);
+
+    Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
+
+    ContainerLaunchContext masterContainerContext = BuilderUtils.newContainerLaunchContext(
+        null, commonContainerLaunchContext.getUser(),
+        resource, commonContainerLaunchContext.getLocalResources(), myEnv, commands,
+        myServiceData, null, new HashMap<ApplicationAccessType, String>(2));
+
+    appContext.setAMContainerSpec(masterContainerContext);
+
+    LOG.info("Submitting QueryMaster to ResourceManager");
+    yarnClient.submitApplication(appContext);
+
+    ApplicationReport appReport = monitorApplication(appId, EnumSet.of(YarnApplicationState.ACCEPTED));
+    ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
+
+    LOG.info("Launching QueryMaster with appAttemptId: " + attemptId);
+
+    return attemptId;
+  }
+
+  private ApplicationReport monitorApplication(ApplicationId appId,
+                                               Set<YarnApplicationState> finalState) throws YarnRemoteException {
+
+    long sleepTime = 100;
+    int count = 1;
+    while (true) {
+      // Get application report for the appId we are interested in
+      ApplicationReport report = yarnClient.getApplicationReport(appId);
+
+      LOG.info("Got application report from ASM for" + ", appId="
+          + appId.getId() + ", appAttemptId="
+          + report.getCurrentApplicationAttemptId() + ", clientToken="
+          + report.getClientToken() + ", appDiagnostics="
+          + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
+          + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
+          + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
+          + ", yarnAppState=" + report.getYarnApplicationState().toString()
+          + ", distributedFinalState="
+          + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
+          + report.getTrackingUrl() + ", appUser=" + report.getUser());
+
+      YarnApplicationState state = report.getYarnApplicationState();
+      if (finalState.contains(state)) {
+        return report;
+      }
+      try {
+        Thread.sleep(sleepTime);
+        sleepTime = count * 100;
+        if(count < 10) {
+          count++;
+        }
+      } catch (InterruptedException e) {
+        //LOG.debug("Thread sleep in monitoring loop interrupted");
+      }
+    }
+  }
+
+  public boolean isQueryMasterStopped(QueryId queryId) {
+    ApplicationId appId = ApplicationIdUtils.queryIdToAppId(queryId);
+    try {
+      ApplicationReport report = yarnClient.getApplicationReport(appId);
+      YarnApplicationState state = report.getYarnApplicationState();
+      return EnumSet.of(
+          YarnApplicationState.FINISHED,
+          YarnApplicationState.KILLED,
+          YarnApplicationState.FAILED).contains(state);
+    } catch (YarnRemoteException e) {
+      LOG.error(e.getMessage(), e);
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
new file mode 100644
index 0000000..5fe1b74
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+
+public class ApplicationIdUtils {
+
+  public static ApplicationAttemptId createApplicationAttemptId(QueryId queryId) {
+    return BuilderUtils.newApplicationAttemptId(queryIdToAppId(queryId), 1);
+  }
+
+  public static ApplicationId queryIdToAppId(QueryId queryId) {
+    return BuilderUtils.newApplicationId(Long.parseLong(queryId.getId()), queryId.getSeq());
+  }
+
+  public static QueryId appIdToQueryId(YarnProtos.ApplicationIdProto appId) {
+    return QueryIdFactory.newQueryId(appId.getClusterTimestamp(), appId.getId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
index f185cc6..b057561 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
@@ -19,27 +19,24 @@
 package org.apache.tajo.webapp;
 
 import org.apache.hadoop.conf.Configuration;
-import org.mortbay.jetty.Connector;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.master.TajoMaster;
+import org.mortbay.jetty.Connector;
 
 import java.io.IOException;
 
 public class StaticHttpServer extends HttpServer {
   private static StaticHttpServer instance = null;
-  private TajoMaster master = null;
-  
-  private StaticHttpServer(TajoMaster master , String name, String bindAddress, int port,
+
+  private StaticHttpServer(Object containerObject , String name, String bindAddress, int port,
       boolean findPort, Connector connector, Configuration conf,
       String[] pathSpecs) throws IOException {
     super( name, bindAddress, port, findPort, connector, conf, pathSpecs);
-    this.master = master;
   }
   public static StaticHttpServer getInstance() {
     return instance;
   }
-  public static StaticHttpServer getInstance( TajoMaster master, String name,
+  public static StaticHttpServer getInstance(Object containerObject, String name,
       String bindAddress, int port, boolean findPort, Connector connector,
       TajoConf conf,
       String[] pathSpecs) throws IOException {
@@ -49,19 +46,16 @@ public class StaticHttpServer extends HttpServer {
         addr = conf.getVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS).split(":")[0];
       }
       
-      instance = new StaticHttpServer(master, name, addr, port,
+      instance = new StaticHttpServer(containerObject, name, addr, port,
           findPort, connector, conf, pathSpecs);
-      instance.setAttribute("tajo.master", master);
-      instance.setAttribute("tajo.master.addr", addr);
-      instance.setAttribute("tajo.master.conf", conf);
-      instance.setAttribute("tajo.master.starttime", System.currentTimeMillis());
+      instance.setAttribute("tajo.info.server.object", containerObject);
+      instance.setAttribute("tajo.info.server.addr", addr);
+      instance.setAttribute("tajo.info.server.conf", conf);
+      instance.setAttribute("tajo.info.server.starttime", System.currentTimeMillis());
     }
     return instance;
   }
-  public TajoMaster getMaster() {
-    
-    return this.master;
-  }
+
   public void set(String name, Object object) {
     instance.setAttribute(name, object);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
new file mode 100644
index 0000000..ebc3f08
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.master.ContainerProxy;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class AbstractResourceAllocator extends CompositeService implements ResourceAllocator {
+  private int minCapability;
+  private int maxCapability;
+  private int numCluster;
+
+  private Map<ContainerId, ContainerProxy> containers = new HashMap<ContainerId, ContainerProxy>();
+
+  public AbstractResourceAllocator() {
+    super(AbstractResourceAllocator.class.getName());
+  }
+
+  public void addContainer(ContainerId cId, ContainerProxy container) {
+    containers.put(cId, container);
+  }
+
+  public void removeContainer(ContainerId cId) {
+    containers.remove(cId);
+  }
+
+  public boolean containsContainer(ContainerId cId) {
+    return containers.containsKey(cId);
+  }
+
+  public ContainerProxy getContainer(ContainerId cId) {
+    return containers.get(cId);
+  }
+
+  public Map<ContainerId, ContainerProxy> getContainers() {
+    return containers;
+  }
+
+  public void setMaxContainerCapability(int capability) {
+    this.maxCapability = capability;
+  }
+
+  public int getMaxContainerCapability() {
+    return this.maxCapability;
+  }
+
+  public void setMinContainerCapability(int capability) {
+    this.minCapability = capability;
+  }
+
+  public int getNumClusterNode() {
+    return numCluster;
+  }
+
+  public void setNumClusterNodes(int num) {
+    numCluster = num;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
new file mode 100644
index 0000000..108c7b7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+
+public interface ResourceAllocator {
+  public void allocateTaskWorker();
+  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
new file mode 100644
index 0000000..3264f55
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -0,0 +1,346 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.ContainerProxy;
+import org.apache.tajo.master.TajoContainerProxy;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+import org.apache.tajo.master.TaskRunnerLauncher;
+import org.apache.tajo.master.event.ContainerAllocationEvent;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.master.rm.TajoWorkerContainer;
+import org.apache.tajo.master.rm.TajoWorkerContainerId;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.CallFuture2;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class TajoResourceAllocator extends AbstractResourceAllocator {
+  private static final Log LOG = LogFactory.getLog(TajoResourceAllocator.class);
+
+  static AtomicInteger containerIdSeq = new AtomicInteger(0);
+  private TajoConf tajoConf;
+  private QueryMasterTask.QueryContext queryContext;
+  private final ExecutorService executorService;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  public TajoResourceAllocator(QueryMasterTask.QueryContext queryContext) {
+    this.queryContext = queryContext;
+    executorService = Executors.newFixedThreadPool(
+        queryContext.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+  }
+
+  @Override
+  public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerIdProto) {
+    TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+    ApplicationAttemptId appAttemptId = new ApplicationAttemptIdPBImpl(containerIdProto.getAppAttemptId());
+    containerId.setApplicationAttemptId(appAttemptId);
+    containerId.setId(containerIdProto.getId());
+    return containerId;
+  }
+
+  @Override
+  public void allocateTaskWorker() {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    tajoConf = (TajoConf)conf;
+
+    queryContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, new TajoTaskRunnerLauncher());
+//
+    queryContext.getDispatcher().register(ContainerAllocatorEventType.class, new TajoWorkerAllocationHandler());
+
+    super.init(conf);
+  }
+
+  @Override
+  public void stop() {
+    if(stopped.get()) {
+      return;
+    }
+    stopped.set(true);
+    executorService.shutdownNow();
+
+    while(!executorService.isTerminated()) {
+      LOG.info("====>executorService.isTerminated:" + executorService.isTerminated() + "," +
+          executorService.isShutdown());
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+      }
+    }
+    Map<ContainerId, ContainerProxy> containers = queryContext.getResourceAllocator().getContainers();
+    for(ContainerProxy eachProxy: containers.values()) {
+      try {
+        eachProxy.stopContainer();
+      } catch (Exception e) {
+      }
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  final public static FsPermission QUERYCONF_FILE_PERMISSION =
+      FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+  private static void writeConf(Configuration conf, Path queryConfFile)
+      throws IOException {
+    // Write job file to Tajo's fs
+    FileSystem fs = queryConfFile.getFileSystem(conf);
+    FSDataOutputStream out =
+        FileSystem.create(fs, queryConfFile,
+            new FsPermission(QUERYCONF_FILE_PERMISSION));
+    try {
+      conf.writeXml(out);
+    } finally {
+      out.close();
+    }
+  }
+
+  class TajoTaskRunnerLauncher implements TaskRunnerLauncher {
+    @Override
+    public void handle(TaskRunnerGroupEvent event) {
+      if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_LAUNCH) {
+        launchTaskRunners(event.getExecutionBlockId(), event.getContainers());
+      } else if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_CLEANUP) {
+        stopContainers(event.getContainers());
+      }
+    }
+  }
+
+  private void launchTaskRunners(ExecutionBlockId executionBlockId, Collection<Container> containers) {
+    FileSystem fs = null;
+
+    QueryConf queryConf = queryContext.getConf();
+    LOG.info("defaultFS: " + queryConf.get("fs.default.name"));
+    LOG.info("defaultFS: " + queryConf.get("fs.defaultFS"));
+    try {
+      fs = FileSystem.get(queryConf);
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    try {
+      // TODO move to tajo temp
+      Path warehousePath = new Path(queryConf.getVar(TajoConf.ConfVars.ROOT_DIR), TajoConstants.WAREHOUSE_DIR);
+      Path queryConfPath = new Path(warehousePath, executionBlockId.getQueryId().toString());
+      queryConfPath = new Path(queryConfPath, QueryConf.FILENAME);
+
+      if(!fs.exists(queryConfPath)){
+        LOG.info("Writing a QueryConf to HDFS and add to local environment, outputPath=" + queryConf.getOutputPath());
+        writeConf(queryConf, queryConfPath);
+      } else {
+        LOG.warn("QueryConf already exist. path: "  + queryConfPath.toString());
+      }
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+    //Query in standby mode doesn't need launch Worker.
+    //But, Assign ExecutionBlock to assigned tajo worker
+    for(Container eachContainer: containers) {
+      TajoContainerProxy containerProxy = new TajoContainerProxy(queryContext, tajoConf,
+          eachContainer, executionBlockId);
+      executorService.submit(new LaunchRunner(eachContainer.getId(), containerProxy));
+    }
+  }
+
+  protected class LaunchRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+      this.proxy = proxy;
+      this.id = id;
+    }
+    @Override
+    public void run() {
+      proxy.launch(null);
+      LOG.info("ContainerProxy started:" + id);
+    }
+  }
+
+  private void stopContainers(Collection<Container> containers) {
+    for (Container container : containers) {
+      final ContainerProxy proxy = queryContext.getResourceAllocator().getContainer(container.getId());
+      executorService.submit(new StopContainerRunner(container.getId(), proxy));
+    }
+  }
+
+  private class StopContainerRunner implements Runnable {
+    private final ContainerProxy proxy;
+    private final ContainerId id;
+    public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+      this.id = id;
+      this.proxy = proxy;
+    }
+
+    @Override
+    public void run() {
+      LOG.info("ContainerProxy stopped:" + id + "," + proxy.getId());
+      proxy.stopContainer();
+    }
+  }
+
+  class TajoWorkerAllocationHandler implements EventHandler<ContainerAllocationEvent> {
+    @Override
+    public void handle(ContainerAllocationEvent event) {
+      executorService.submit(new TajoWorkerAllocationThread(event));
+    }
+  }
+
+  class TajoWorkerAllocationThread extends Thread {
+    ContainerAllocationEvent event;
+    TajoWorkerAllocationThread(ContainerAllocationEvent event) {
+      this.event = event;
+    }
+
+    @Override
+    public void run() {
+      LOG.info("======> Start TajoWorkerAllocationThread");
+      CallFuture2<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack =
+          new CallFuture2<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();
+
+      queryContext.getQueryMasterContext().getWorkerContext().
+          getTajoMasterRpcClient().allocateWorkerResources(null, request, callBack);
+
+      int numAllocatedWorkers = 0;
+      while(!stopped.get()) {
+        TajoMasterProtocol.WorkerResourceAllocationResponse response = null;
+        try {
+          response = callBack.get(3, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+          if(stopped.get()) {
+            break;
+          }
+        } catch (TimeoutException e) {
+          LOG.info("No available worker resource for " + event.getExecutionBlockId());
+          continue;
+        }
+
+        List<String> workerHosts = response.getAllocatedWorksList();
+        ExecutionBlockId executionBlockId = event.getExecutionBlockId();
+
+        List<Container> containers = new ArrayList<Container>();
+        for(String eachWorker: workerHosts) {
+          TajoWorkerContainer container = new TajoWorkerContainer();
+          NodeIdPBImpl nodeId = new NodeIdPBImpl();
+          String[] tokens = eachWorker.split(":");
+
+          nodeId.setHost(tokens[0]);
+          nodeId.setPort(Integer.parseInt(tokens[1]));
+
+          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.setPorts(new int[]{nodeId.getPort()});
+          workerResource.setMemoryMBSlots(requiredMemoryMBSlot);
+          workerResource.setDiskSlots(requiredDiskSlots);
+
+          container.setWorkerResource(workerResource);
+
+          containers.add(container);
+        }
+
+        SubQueryState state = queryContext.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(queryContext, executionBlockId, workerResources);
+          } catch (Exception e) {
+            LOG.error(e.getMessage(), e);
+          }
+          return;
+        }
+
+        if (workerHosts.size() > 0) {
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("SubQueryContainerAllocationEvent fire:" + executionBlockId);
+          }
+          queryContext.getEventHandler().handle(new SubQueryContainerAllocationEvent(executionBlockId, containers));
+        }
+        numAllocatedWorkers += workerHosts.size();
+        if(numAllocatedWorkers >= event.getRequiredNum()) {
+          break;
+        }
+      }
+      LOG.info("======> Stop TajoWorkerAllocationThread");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
new file mode 100644
index 0000000..3b116f2
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -0,0 +1,424 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryMaster;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.pullserver.TajoPullServerService;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.ProtoAsyncRpcClient;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.webapp.StaticHttpServer;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TajoWorker extends CompositeService {
+  public static PrimitiveProtos.BoolProto TRUE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+  public static PrimitiveProtos.BoolProto FALSE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+  private static final Log LOG = LogFactory.getLog(TajoWorker.class);
+
+  private TajoConf tajoConf;
+
+  private StaticHttpServer webServer;
+
+  private TajoWorkerClientService tajoWorkerClientService;
+
+  private TajoWorkerManagerService tajoWorkerManagerService;
+
+  //to TajoMaster
+  private ProtoAsyncRpcClient tajoMasterRpc;
+
+  private TajoMasterProtocol.TajoMasterProtocolService tajoMasterRpcClient;
+
+  private WorkerContext workerContext;
+
+  private TaskRunnerManager taskRunnerManager;
+
+  private TajoPullServerService pullService;
+
+  private String daemonMode;
+
+  private WorkerHeartbeatThread workerHeartbeatThread;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  public TajoWorker(String daemonMode) throws Exception {
+    super(TajoWorker.class.getName());
+    this.daemonMode = daemonMode;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+
+    this.tajoConf = (TajoConf)conf;
+    RackResolver.init(tajoConf);
+
+    workerContext = new WorkerContext();
+
+    String resourceManagerClassName = conf.get("tajo.resource.manager",
+        TajoWorkerResourceManager.class.getCanonicalName());
+
+    boolean randomPort = true;
+    if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+      randomPort = false;
+    }
+    int infoPort = tajoConf.getInt("tajo.worker.info.port", 8090);
+    int clientPort = tajoConf.getInt("tajo.worker.client.rpc.port", 8091);
+    int managerPort = tajoConf.getInt("tajo.worker.manager.rpc.port", 8092);
+
+    if(randomPort) {
+      clientPort = 0;
+      managerPort = 0;
+      tajoConf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname, 0);
+      //infoPort = 0;
+    }
+    try {
+      //TODO WebServer port configurable
+      webServer = StaticHttpServer.getInstance(this, "admin", null, infoPort,
+          true, null, tajoConf, null);
+      webServer.start();
+    } catch (Exception e) {
+      LOG.error("Can' start info http server:" + e.getMessage(), e);
+    }
+
+    if(!"qm".equals(daemonMode)) {
+      taskRunnerManager = new TaskRunnerManager(workerContext);
+      addService(taskRunnerManager);
+    }
+
+    if(workerContext.isStandbyMode()) {
+      pullService = new TajoPullServerService();
+      addService(pullService);
+    }
+
+    if(!"tr".equals(daemonMode)) {
+      tajoWorkerClientService = new TajoWorkerClientService(workerContext, clientPort);
+      addService(tajoWorkerClientService);
+
+      tajoWorkerManagerService = new TajoWorkerManagerService(workerContext, managerPort);
+      addService(tajoWorkerManagerService);
+      LOG.info("====> Tajo worker started: mode=" + daemonMode + ", clientPort=" + clientPort + ", managerPort=" + managerPort);
+    } else {
+      LOG.info("====> Tajo worker started: mode=" + daemonMode);
+    }
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(stopped.get()) {
+      return;
+    }
+    stopped.set(true);
+    if(webServer != null) {
+      try {
+        webServer.stop();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      }
+    }
+    if(workerHeartbeatThread != null) {
+      workerHeartbeatThread.interrupt();
+    }
+
+//    try {
+//      FileSystem.closeAll();
+//    } catch (IOException e) {
+//      LOG.error(e.getMessage(), e);
+//    }
+    if(tajoMasterRpc != null) {
+      tajoMasterRpc.close();
+    }
+//    for(Service eachService: getServices()) {
+//      System.out.println("Service:" + eachService);
+//    }
+    super.stop();
+    LOG.info("TajoWorker main thread exiting");
+  }
+
+  public class WorkerContext {
+    public QueryMaster getQueryMaster() {
+      return tajoWorkerManagerService.getQueryMaster();
+    }
+
+    public TajoWorkerManagerService getTajoWorkerManagerService() {
+      return tajoWorkerManagerService;
+    }
+
+    public TajoWorkerClientService getTajoWorkerClientService() {
+      return tajoWorkerClientService;
+    }
+
+    public TajoMasterProtocol.TajoMasterProtocolService getTajoMasterRpcClient() {
+      return tajoMasterRpcClient;
+    }
+
+    public TaskRunnerManager getTaskRunnerManager() {
+      return taskRunnerManager;
+    }
+
+    public TajoPullServerService getPullService() {
+      return pullService;
+    }
+
+    public void stopWorker(boolean force) {
+      stop();
+      if(force) {
+        System.exit(0);
+      }
+    }
+
+    public boolean isStandbyMode() {
+      return !"qm".equals(daemonMode) && !"tr".equals(daemonMode);
+    }
+  }
+
+  public void stopWorkerForce() {
+    stop();
+  }
+
+  private void setWorkerMode(String[] params) {
+    if("qm".equals(daemonMode)) {
+      //QueryMaster mode
+      String tajoMasterAddress = params[2];
+
+      LOG.info("Init TajoMaster connection to:" + tajoMasterAddress);
+      InetSocketAddress addr = NetUtils.createSocketAddr(tajoMasterAddress);
+      try {
+        tajoMasterRpc = new ProtoAsyncRpcClient(TajoMasterProtocol.class, addr);
+        tajoMasterRpcClient = tajoMasterRpc.getStub();
+      } catch (Exception e) {
+        LOG.error("Can't connect to TajoMaster[" + addr + "], " + e.getMessage(), e);
+      }
+
+      QueryId queryId = TajoIdUtils.parseQueryId(params[1]);
+      tajoWorkerManagerService.getQueryMaster().reportQueryStatusToQueryMaster(
+          queryId, TajoProtos.QueryState.QUERY_MASTER_LAUNCHED);
+    } else if("tr".equals(daemonMode)) {
+      //TaskRunner mode
+      taskRunnerManager.startTask(params);
+    } else {
+      //Standby mode
+      String tajoMasterAddress = tajoConf.get("tajo.master.manager.addr");
+      LOG.info("Init TajoMaster connection to:" + tajoMasterAddress);
+      InetSocketAddress addr = NetUtils.createSocketAddr(tajoMasterAddress);
+      try {
+        tajoMasterRpc = new ProtoAsyncRpcClient(TajoMasterProtocol.class, addr);
+        tajoMasterRpcClient = tajoMasterRpc.getStub();
+      } catch (Exception e) {
+        LOG.error("Can't connect to TajoMaster[" + addr + "], " + e.getMessage(), e);
+      }
+      workerHeartbeatThread = new WorkerHeartbeatThread();
+      workerHeartbeatThread.start();
+    }
+  }
+
+  class WorkerHeartbeatThread extends Thread {
+    TajoMasterProtocol.ServerStatusProto.System systemInfo;
+    List<TajoMasterProtocol.ServerStatusProto.Disk> diskInfos =
+        new ArrayList<TajoMasterProtocol.ServerStatusProto.Disk>();
+    int workerDiskSlots;
+    List<File> mountPaths;
+
+    public WorkerHeartbeatThread() {
+      int workerMemoryMBSlots;
+      int workerCpuCoreSlots;
+
+      boolean useSystemInfo = tajoConf.getBoolean("tajo.worker.slots.use.os.info", true);
+
+      try {
+        mountPaths = getMountPath();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      }
+
+      if(useSystemInfo) {
+        float memoryRatio = tajoConf.getFloat("tajo.worker.slots.os.memory.ratio", 0.8f);
+        workerMemoryMBSlots = getTotalMemoryMB();
+        workerMemoryMBSlots = (int)((float)(workerMemoryMBSlots) * memoryRatio);
+        workerCpuCoreSlots = Runtime.getRuntime().availableProcessors();
+        if(mountPaths == null) {
+          workerDiskSlots = 2;
+        } else {
+          workerDiskSlots = mountPaths.size();
+        }
+      } else {
+        workerMemoryMBSlots = tajoConf.getInt("tajo.worker.slots.memoryMB", 2048);
+        workerDiskSlots = tajoConf.getInt("tajo.worker.slots.disk", 2);
+        workerCpuCoreSlots = tajoConf.getInt("tajo.worker.slots.cpu.core", 4);
+      }
+
+      workerDiskSlots = workerDiskSlots * tajoConf.getInt("tajo.worker.slots.disk.concurrency", 4);
+
+      systemInfo = TajoMasterProtocol.ServerStatusProto.System.newBuilder()
+          .setAvailableProcessors(workerCpuCoreSlots)
+          .setFreeMemoryMB(0)
+          .setMaxMemoryMB(0)
+          .setTotalMemoryMB(workerMemoryMBSlots)
+          .build();
+    }
+
+    public void run() {
+      LOG.info("Worker Resource Heartbeat Thread start.");
+      int sendDiskInfoCount = 0;
+      while(true) {
+        if(sendDiskInfoCount == 0 && mountPaths != null) {
+          for(File eachFile: mountPaths) {
+            diskInfos.clear();
+            diskInfos.add(TajoMasterProtocol.ServerStatusProto.Disk.newBuilder()
+                .setAbsolutePath(eachFile.getAbsolutePath())
+                .setTotalSpace(eachFile.getTotalSpace())
+                .setFreeSpace(eachFile.getFreeSpace())
+                .setUsableSpace(eachFile.getUsableSpace())
+                .build());
+          }
+        }
+        TajoMasterProtocol.ServerStatusProto serverStatus = TajoMasterProtocol.ServerStatusProto.newBuilder()
+            .addAllDisk(diskInfos)
+            .setRunningTaskNum(0)   //TODO
+            .setSystem(systemInfo)
+            .setDiskSlots(workerDiskSlots)
+            .build();
+
+        TajoMasterProtocol.TajoHeartbeat heartbeatProto = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+            .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+            .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+            .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+            .setServerStatus(serverStatus)
+            .build();
+
+        workerContext.getTajoMasterRpcClient().heartbeat(null, heartbeatProto, NullCallback.get());
+        try {
+          Thread.sleep(10 * 1000);
+        } catch (InterruptedException e) {
+          break;
+        }
+        sendDiskInfoCount++;
+
+        if(sendDiskInfoCount > 10) {
+          sendDiskInfoCount = 0;
+        }
+      }
+
+      LOG.info("Worker Resource Heartbeat Thread stopped.");
+    }
+  }
+
+  private class ShutdownHook implements Runnable {
+    @Override
+    public void run() {
+      if(!stopped.get()) {
+        LOG.info("============================================");
+        LOG.info("TajoWorker received SIGINT Signal");
+        LOG.info("============================================");
+        stop();
+      }
+    }
+  }
+
+  public void startWorker(TajoConf tajoConf, String[] args) {
+    init(tajoConf);
+    start();
+    setWorkerMode(args);
+  }
+
+  public static List<File> getMountPath() throws Exception {
+    BufferedReader mountOutput = null;
+    try {
+      Process mountProcess = Runtime.getRuntime ().exec("mount");
+      mountOutput = new BufferedReader(new InputStreamReader(mountProcess.getInputStream()));
+      List<File> mountPaths = new ArrayList<File>();
+      while (true) {
+        String line = mountOutput.readLine();
+        if (line == null) {
+          break;
+        }
+
+        System.out.println(line);
+
+        int indexStart = line.indexOf(" on /");
+        int indexEnd = line.indexOf(" ", indexStart + 4);
+
+        mountPaths.add(new File(line.substring (indexStart + 4, indexEnd)));
+      }
+      return mountPaths;
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw e;
+    } finally {
+      if(mountOutput != null) {
+        mountOutput.close();
+      }
+    }
+  }
+
+  public static int getTotalMemoryMB() {
+    com.sun.management.OperatingSystemMXBean bean =
+        (com.sun.management.OperatingSystemMXBean)
+            java.lang.management.ManagementFactory.getOperatingSystemMXBean();
+    long max = bean.getTotalPhysicalMemorySize();
+    return ((int)(max/1024));
+  }
+
+  public static void main(String[] args) throws Exception {
+    StringUtils.startupShutdownMessage(TajoWorker.class, args, LOG);
+
+    if(args.length < 1) {
+      LOG.error("Wrong startup params");
+      System.exit(-1);
+    }
+
+    String workerMode = args[0];
+
+    try {
+      TajoWorker tajoWorker = new TajoWorker(workerMode);
+      tajoWorker.startWorker(new TajoConf(new YarnConfiguration()), args);
+    } catch (Throwable t) {
+      LOG.fatal("Error starting TajoWorker", t);
+      System.exit(-1);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
new file mode 100644
index 0000000..3969b36
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.QueryMasterClientProtocol;
+import org.apache.tajo.master.querymaster.Query;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.rpc.ProtoBlockingRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+
+public class TajoWorkerClientService extends AbstractService {
+  private static final Log LOG = LogFactory.getLog(TajoWorkerClientService.class);
+  private final PrimitiveProtos.BoolProto BOOL_TRUE =
+          PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+  private final PrimitiveProtos.BoolProto BOOL_FALSE =
+          PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+  private ProtoBlockingRpcServer rpcServer;
+  private InetSocketAddress bindAddr;
+  private String addr;
+  private int port;
+  private Configuration conf;
+  private TajoWorker.WorkerContext workerContext;
+  private TajoWorkerClientProtocolServiceHandler serviceHandler;
+
+  public TajoWorkerClientService(TajoWorker.WorkerContext workerContext, int port) {
+    super(TajoWorkerClientService.class.getName());
+
+    this.port = port;
+    this.workerContext = workerContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.conf = conf;
+    this.serviceHandler = new TajoWorkerClientProtocolServiceHandler();
+
+    // init RPC Server in constructor cause Heartbeat Thread use bindAddr
+    // Setup RPC server
+    try {
+      // TODO initial port num is value of config and find unused port with sequence
+      InetSocketAddress initIsa = new InetSocketAddress("0.0.0.0", port);
+      if (initIsa.getAddress() == null) {
+        throw new IllegalArgumentException("Failed resolve of " + initIsa);
+      }
+
+      // TODO blocking/non-blocking??
+      this.rpcServer = new ProtoBlockingRpcServer(QueryMasterClientProtocol.class, serviceHandler, initIsa);
+      this.rpcServer.start();
+
+      this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+      this.port = bindAddr.getPort();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    // Get the master address
+    LOG.info(TajoWorkerClientService.class.getSimpleName() + " is bind to " + addr);
+    //queryConf.setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    LOG.info("====> TajoWorkerClientService stopping");
+    if(rpcServer != null) {
+      rpcServer.shutdown();
+    }
+    LOG.info("====> TajoWorkerClientService stopped");
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddr() {
+    return bindAddr;
+  }
+
+  public class TajoWorkerClientProtocolServiceHandler
+          implements QueryMasterClientProtocol.QueryMasterClientProtocolService.BlockingInterface {
+    @Override
+    public PrimitiveProtos.BoolProto updateSessionVariables(
+            RpcController controller,
+            ClientProtos.UpdateSessionVariableRequest request) throws ServiceException {
+      return null;
+    }
+
+    @Override
+    public ClientProtos.GetQueryResultResponse getQueryResult(
+            RpcController controller,
+            ClientProtos.GetQueryResultRequest request) throws ServiceException {
+      QueryId queryId = new QueryId(request.getQueryId());
+      Query query = workerContext.getQueryMaster().getQuery(queryId);
+
+      ClientProtos.GetQueryResultResponse.Builder builder = ClientProtos.GetQueryResultResponse.newBuilder();
+
+      if(query == null) {
+        builder.setErrorMessage("No Query for " + queryId);
+      } else {
+        switch (query.getState()) {
+          case QUERY_SUCCEEDED:
+            builder.setTableDesc((CatalogProtos.TableDescProto)query.getResultDesc().getProto());
+            break;
+          case QUERY_FAILED:
+          case QUERY_ERROR:
+            builder.setErrorMessage("Query " + queryId + " is failed");
+          default:
+            builder.setErrorMessage("Query " + queryId + " is still running");
+        }
+      }
+      return builder.build();
+    }
+
+    @Override
+    public ClientProtos.GetQueryStatusResponse getQueryStatus(
+            RpcController controller,
+            ClientProtos.GetQueryStatusRequest request) throws ServiceException {
+      ClientProtos.GetQueryStatusResponse.Builder builder
+              = ClientProtos.GetQueryStatusResponse.newBuilder();
+      QueryId queryId = new QueryId(request.getQueryId());
+
+      builder.setQueryId(request.getQueryId());
+
+      if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+        builder.setResultCode(ClientProtos.ResultCode.OK);
+        builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
+      } else {
+        QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId);
+        builder.setResultCode(ClientProtos.ResultCode.OK);
+        builder.setQueryMasterHost(bindAddr.getHostName());
+        builder.setQueryMasterPort(bindAddr.getPort());
+
+        if (queryMasterTask != null) {
+          queryMasterTask.touchSessionTime();
+          Query query = queryMasterTask.getQuery();
+
+          builder.setState(query.getState());
+          builder.setProgress(query.getProgress());
+          builder.setSubmitTime(query.getAppSubmitTime());
+          builder.setInitTime(query.getInitializationTime());
+          builder.setHasResult(!queryMasterTask.getQueryContext().isCreateTableQuery());
+          if (query.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+            builder.setFinishTime(query.getFinishTime());
+          } else {
+            builder.setFinishTime(System.currentTimeMillis());
+          }
+        } else {
+          builder.setState(TajoProtos.QueryState.QUERY_NOT_ASSIGNED);
+        }
+      }
+
+      return builder.build();
+    }
+
+    @Override
+    public PrimitiveProtos.BoolProto killQuery (
+            RpcController controller,
+            TajoIdProtos.QueryIdProto request) throws ServiceException {
+      final QueryId queryId = new QueryId(request);
+      LOG.info("Stop Query:" + queryId);
+      Thread t = new Thread() {
+        public void run() {
+//          try {
+//            Thread.sleep(1000);   //wait tile return to rpc response
+//          } catch (InterruptedException e) {
+//          }
+          workerContext.getQueryMaster().getContext().stopQuery(queryId);
+        }
+      };
+      t.start();
+      return BOOL_TRUE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
new file mode 100644
index 0000000..a48339a
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TaskSchedulerImpl;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.querymaster.QueryMaster;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.rpc.ProtoAsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+
+public class TajoWorkerManagerService extends CompositeService
+    implements TajoWorkerProtocol.TajoWorkerProtocolService.Interface {
+  private static final Log LOG = LogFactory.getLog(TajoWorkerManagerService.class.getName());
+
+  private ProtoAsyncRpcServer rpcServer;
+  private InetSocketAddress bindAddr;
+  private String addr;
+  private int port;
+
+  private QueryMaster queryMaster;
+
+  private TajoWorker.WorkerContext workerContext;
+
+  public TajoWorkerManagerService(TajoWorker.WorkerContext workerContext, int port) {
+    super(TajoWorkerManagerService.class.getName());
+    this.workerContext = workerContext;
+    this.port = port;
+  }
+
+  public QueryMaster getQueryMaster() {
+    return queryMaster;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    try {
+      // Setup RPC server
+      InetSocketAddress initIsa =
+          new InetSocketAddress("0.0.0.0", port);
+      if (initIsa.getAddress() == null) {
+        throw new IllegalArgumentException("Failed resolve of " + initIsa);
+      }
+
+      this.rpcServer = new ProtoAsyncRpcServer(TajoWorkerProtocol.class, this, initIsa);
+      this.rpcServer.start();
+
+      this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+      this.port = bindAddr.getPort();
+
+      queryMaster = new QueryMaster(workerContext);
+      addService(queryMaster);
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    // Get the master address
+    LOG.info("TajoWorkerManagerService is bind to " + addr);
+    ((TajoConf)conf).setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(rpcServer != null) {
+      rpcServer.shutdown();
+    }
+    LOG.info("TajoWorkerManagerService stopped");
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddr() {
+    return bindAddr;
+  }
+
+  public String getHostAndPort() {
+    return bindAddr.getHostName() + ":" + bindAddr.getPort();
+  }
+
+  @Override
+  public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestProto request,
+                      RpcCallback<TajoWorkerProtocol.QueryUnitRequestProto> done) {
+    try {
+      ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId());
+      QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(ebId.getQueryId());
+      ContainerId cid =
+          queryMasterTask.getQueryContext().getResourceAllocator().makeContainerId(request.getContainerId());
+
+      if(queryMasterTask == null || queryMasterTask.isStopped()) {
+        LOG.info("====>getTask:" + cid + ", ebId:" + ebId + ", but query is finished.");
+        done.run(TaskSchedulerImpl.stopTaskRunnerReq);
+      } else {
+        LOG.info("====>getTask:" + cid + ", ebId:" + ebId);
+        queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(cid, ebId, done));
+      }
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  @Override
+  public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request,
+                           RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+          new QueryId(request.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+      queryMasterTask.getEventHandler().handle(
+          new TaskAttemptStatusUpdateEvent(new QueryUnitAttemptId(request.getId()), request));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void ping(RpcController controller,
+                   TajoIdProtos.QueryUnitAttemptIdProto attemptId,
+                   RpcCallback<PrimitiveProtos.BoolProto> done) {
+    // TODO - to be completed
+//      QueryUnitAttemptId attemptId = new QueryUnitAttemptId(attemptIdProto);
+//    context.getQuery(attemptId.getQueryId()).getSubQuery(attemptId.getExecutionBlockId()).
+//        getQueryUnit(attemptId.getQueryUnitId()).getAttempt(attemptId).
+//        resetExpireTime();
+    done.run(TajoWorker.TRUE_PROTO);
+  }
+
+  @Override
+  public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErrorReport report,
+                         RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+          new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+      queryMasterTask.getEventHandler().handle(new TaskFatalErrorEvent(report));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionReport report,
+                   RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+          new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+      queryMasterTask.getEventHandler().handle(new TaskCompletionEvent(report));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void executeQuery(RpcController controller,
+                           TajoWorkerProtocol.QueryExecutionRequestProto request,
+                           RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      QueryId queryId = new QueryId(request.getQueryId());
+      LOG.info("====>Receive executeQuery request:" + queryId);
+      queryMaster.handle(new QueryStartEvent(queryId, request.getLogicalPlanJson().getValue()));
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+
+  @Override
+  public void executeExecutionBlock(RpcController controller,
+                                    TajoWorkerProtocol.RunExecutionBlockRequestProto request,
+                                    RpcCallback<PrimitiveProtos.BoolProto> done) {
+    try {
+      String[] params = new String[7];
+      params[0] = "standby";  //mode(never used)
+      params[1] = request.getExecutionBlockId();
+      // NodeId has a form of hostname:port.
+      params[2] = request.getNodeId();
+      params[3] = request.getContainerId();
+
+      // QueryMaster's address
+      params[4] = request.getQueryMasterHost();
+      params[5] = String.valueOf(request.getQueryMasterPort());
+      params[6] = request.getQueryOutputPath();
+      workerContext.getTaskRunnerManager().startTask(params);
+      done.run(TajoWorker.TRUE_PROTO);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      done.run(TajoWorker.FALSE_PROTO);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
index 94f1662..f9cc82c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
@@ -37,7 +37,6 @@ import org.apache.tajo.TaskAttemptContext;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.ipc.QueryMasterProtocol.*;
 import org.apache.tajo.engine.exception.UnfinishedTaskException;
 import org.apache.tajo.engine.json.CoreGsonHelper;
 import org.apache.tajo.engine.planner.PlannerUtil;
@@ -45,14 +44,15 @@ import org.apache.tajo.engine.planner.logical.LogicalNode;
 import org.apache.tajo.engine.planner.logical.SortNode;
 import org.apache.tajo.engine.planner.logical.StoreTableNode;
 import org.apache.tajo.engine.planner.physical.PhysicalExec;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.Interface;
+import org.apache.tajo.ipc.TajoWorkerProtocol.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService.Interface;
 import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
 import org.apache.tajo.master.ExecutionBlock.PartitionType;
 import org.apache.tajo.rpc.NullCallback;
 import org.apache.tajo.storage.Fragment;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.storage.TupleComparator;
-import org.apache.tajo.worker.TaskRunner.WorkerContext;
+import org.apache.tajo.util.ApplicationIdUtils;
 
 import java.io.File;
 import java.io.IOException;
@@ -70,7 +70,7 @@ public class Task {
 
   private final QueryConf conf;
   private final FileSystem localFS;
-  private final WorkerContext workerContext;
+  private final TaskRunner.TaskRunnerContext taskRunnerContext;
   private final Interface masterProxy;
   private final LocalDirAllocator lDirAllocator;
   private final QueryUnitAttemptId taskId;
@@ -128,19 +128,19 @@ public class Task {
       };
 
   public Task(QueryUnitAttemptId taskId,
-              final WorkerContext worker, final Interface masterProxy,
+              final TaskRunner.TaskRunnerContext worker, final Interface masterProxy,
               final QueryUnitRequest request) throws IOException {
     this.request = request;
     this.reporter = new Reporter(masterProxy);
     this.reporter.startCommunicationThread();
 
     this.taskId = request.getId();
-    this.conf = worker.getConf();
-    this.workerContext = worker;
+    this.conf = worker.getQueryConf();
+    this.taskRunnerContext = worker;
     this.masterProxy = masterProxy;
     this.localFS = worker.getLocalFS();
     this.lDirAllocator = worker.getLocalDirAllocator();
-    this.taskDir = StorageUtil.concatPath(workerContext.getBaseDir(),
+    this.taskDir = StorageUtil.concatPath(taskRunnerContext.getBaseDir(),
         taskId.getQueryUnitId().getId() + "_" + taskId.getId());
 
     this.context = new TaskAttemptContext(conf, taskId,
@@ -162,7 +162,7 @@ public class Task {
       // where ss is the subquery id associated with this task, and nnnnnn is the task id.
       Path outFilePath = new Path(conf.getOutputPath(),
           OUTPUT_FILE_PREFIX +
-          OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskId.getSubQueryId().getId()) + "-" +
+          OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
           OUTPUT_FILE_FORMAT_TASK.get().format(taskId.getQueryUnitId().getId()));
       LOG.info("Output File Path: " + outFilePath);
       context.setOutputPath(outFilePath);
@@ -184,8 +184,10 @@ public class Task {
       LOG.info("==> Table Id: " + f.getName() + ", url: " + f.getUrls());
     }
     LOG.info("* Local task dir: " + taskDir);
-    LOG.info("* plan:\n");
-    LOG.info(plan.toString());
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("* plan:\n");
+      LOG.debug(plan.toString());
+    }
     LOG.info("==================================");
   }
 
@@ -207,7 +209,6 @@ public class Task {
         }
       }
     }
-
     // for localizing the intermediate data
     localize(request);
   }
@@ -243,7 +244,7 @@ public class Task {
       int i = fetcherRunners.size();
       for (Fragment cache : cached) {
         inFile = new Path(inputTableBaseDir, "in_" + i);
-        workerContext.getDefaultFS().copyToLocalFile(cache.getPath(), inFile);
+        taskRunnerContext.getDefaultFS().copyToLocalFile(cache.getPath(), inFile);
         cache.setPath(inFile);
         i++;
       }
@@ -273,7 +274,7 @@ public class Task {
 
   public void fetch() {
     for (Fetcher f : fetcherRunners) {
-      workerContext.getFetchLauncher().submit(new FetchRunner(context, f));
+      taskRunnerContext.getFetchLauncher().submit(new FetchRunner(context, f));
     }
   }
 
@@ -298,8 +299,8 @@ public class Task {
         // context.getWorkDir() 지우기
         localFS.delete(context.getWorkDir(), true);
         // tasks에서 자기 지우기
-        synchronized (workerContext.getTasks()) {
-          workerContext.getTasks().remove(this.getId());
+        synchronized (taskRunnerContext.getTasks()) {
+          taskRunnerContext.getTasks().remove(this.getId());
         }
       } catch (IOException e) {
         e.printStackTrace();
@@ -312,7 +313,7 @@ public class Task {
 
   public TaskStatusProto getReport() {
     TaskStatusProto.Builder builder = TaskStatusProto.newBuilder();
-    builder.setWorkerName(workerContext.getNodeId());
+    builder.setWorkerName(taskRunnerContext.getNodeId());
     builder.setId(context.getTaskId().getProto())
         .setProgress(context.getProgress()).setState(context.getState());
 
@@ -368,7 +369,7 @@ public class Task {
       }
 
       if (context.getFragmentSize() > 0) {
-        this.executor = workerContext.getTQueryEngine().
+        this.executor = taskRunnerContext.getTQueryEngine().
             createPlan(context, plan);
         this.executor.init();
         while(executor.next() != null && !killed) {
@@ -430,7 +431,7 @@ public class Task {
   }
 
   public void cleanupTask() {
-    workerContext.getTasks().remove(getId());
+    taskRunnerContext.getTasks().remove(getId());
   }
 
   public int hashCode() {
@@ -623,7 +624,7 @@ public class Task {
   String fileCache;
   public String getFileCacheDir() {
     fileCache = USERCACHE + "/" + "hyunsik" + "/" + APPCACHE + "/" +
-        ConverterUtils.toString(taskId.getQueryId().getApplicationId()) +
+        ConverterUtils.toString(ApplicationIdUtils.queryIdToAppId(taskId.getQueryUnitId().getExecutionBlockId().getQueryId())) +
         "/" + "output";
     return fileCache;
   }
@@ -631,7 +632,7 @@ public class Task {
   public static Path getTaskAttemptDir(QueryUnitAttemptId quid) {
     Path workDir =
         StorageUtil.concatPath(
-            quid.getSubQueryId().toString(),
+            quid.getQueryUnitId().getExecutionBlockId().toString(),
             String.valueOf(quid.getQueryUnitId().getId()),
             String.valueOf(quid.getId()));
     return workDir;


[5/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
new file mode 100644
index 0000000..b2c129f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class QueryJobEvent extends AbstractEvent<QueryJobEvent.Type> {
+  private QueryInfo queryInfo;
+
+  public QueryJobEvent(Type type, QueryInfo queryInfo) {
+    super(type);
+
+    this.queryInfo = queryInfo;
+  }
+
+  public QueryInfo getQueryInfo() {
+    return this.queryInfo;
+  }
+
+  public enum Type {
+    QUERY_JOB_START,
+    QUERY_JOB_HEARTBEAT,
+    QUERY_JOB_FINISH,
+    QUERY_MASTER_START,
+    QUERY_MASTER_STOP
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
new file mode 100644
index 0000000..e4d83af
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.WorkerResource;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class QueryJobManager extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(QueryJobManager.class.getName());
+
+  // TajoMaster Context
+  private final TajoMaster.MasterContext masterContext;
+
+  private AsyncDispatcher dispatcher;
+
+  private Map<QueryId, QueryInProgress> runningQueries = new HashMap<QueryId, QueryInProgress>();
+
+  private Map<QueryId, QueryInProgress> finishedQueries = new HashMap<QueryId, QueryInProgress>();
+
+  public QueryJobManager(final TajoMaster.MasterContext masterContext) {
+    super(QueryJobManager.class.getName());
+    this.masterContext = masterContext;
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    try {
+      this.dispatcher = new AsyncDispatcher();
+      addService(this.dispatcher);
+
+      this.dispatcher.register(QueryJobEvent.Type.class, new QueryJobManagerEventHandler());
+    } catch (Exception e) {
+      catchException(null, e);
+    }
+
+    super.init(conf);
+  }
+
+  @Override
+  public void stop() {
+    synchronized(runningQueries) {
+      for(QueryInProgress eachQueryInProgress: runningQueries.values()) {
+        eachQueryInProgress.stop();
+      }
+    }
+    super.stop();
+  }
+
+  @Override
+  public void start() {
+    super.start();
+  }
+
+  public EventHandler getEventHandler() {
+    return dispatcher.getEventHandler();
+  }
+
+  public QueryInfo createNewQueryJob(String sql, LogicalRootNode plan) throws Exception {
+    QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
+    QueryInProgress queryInProgress = new QueryInProgress(masterContext, queryId, sql, plan);
+
+    synchronized(runningQueries) {
+      runningQueries.put(queryId, queryInProgress);
+    }
+
+    addService(queryInProgress);
+    queryInProgress.init(getConfig());
+    queryInProgress.start();
+
+    queryInProgress.startQueryMaster();
+
+    return queryInProgress.getQueryInfo();
+  }
+
+  class QueryJobManagerEventHandler implements EventHandler<QueryJobEvent> {
+    @Override
+    public void handle(QueryJobEvent event) {
+      QueryInProgress queryInProgress = null;
+      synchronized(runningQueries) {
+        queryInProgress = runningQueries.get(event.getQueryInfo().getQueryId());
+        if(queryInProgress == null) {
+          LOG.warn("No query info in running queries.[" + event.getQueryInfo().getQueryId() + "]");
+          return;
+        }
+      }
+      queryInProgress.getEventHandler().handle(event);
+    }
+  }
+
+  public QueryInProgress getQueryInProgress(QueryId queryId) {
+    synchronized(runningQueries) {
+      return runningQueries.get(queryId);
+    }
+  }
+
+  public void stopQuery(QueryId queryId) {
+    LOG.info("====>Stop QueryInProgress:" + queryId);
+    QueryInProgress queryInProgress = getQueryInProgress(queryId);
+    if(queryInProgress != null) {
+      queryInProgress.stop();
+      synchronized(runningQueries) {
+        runningQueries.remove(queryId);
+        finishedQueries.put(queryId, queryInProgress);
+      }
+    } else {
+      LOG.warn("====> No QueryInProgress while query stopping: " + queryId);
+    }
+  }
+
+  private void catchException(QueryId queryId, Exception e) {
+    LOG.error(e.getMessage(), e);
+    QueryInProgress queryInProgress = runningQueries.get(queryId);
+    queryInProgress.catchException(e);
+  }
+
+  public synchronized TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand queryHeartbeat(
+      TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+    QueryInProgress queryInProgress = getQueryInProgress(new QueryId(queryHeartbeat.getQueryId()));
+    if(queryInProgress == null) {
+      return null;
+    }
+
+    QueryInfo queryInfo = makeQueryInfoFromHeartbeat(queryHeartbeat);
+    getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_HEARTBEAT, queryInfo));
+
+    return null;
+  }
+
+  private QueryInfo makeQueryInfoFromHeartbeat(TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+    QueryInfo queryInfo = new QueryInfo(new QueryId(queryHeartbeat.getQueryId()));
+    if(queryHeartbeat.getTajoWorkerHost() != null) {
+      WorkerResource queryMasterResource = new WorkerResource();
+      queryMasterResource.setAllocatedHost(queryHeartbeat.getTajoWorkerHost());
+      queryMasterResource.setPorts(new int[]{queryHeartbeat.getTajoWorkerPort(), queryHeartbeat.getTajoWorkerClientPort()});
+
+      queryInfo.setQueryMasterResource(queryMasterResource);
+    }
+    queryInfo.setLastMessage(queryHeartbeat.getStatusMessage());
+    queryInfo.setQueryState(queryHeartbeat.getState());
+
+    return queryInfo;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
index b66ef68..50ec5be 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -18,142 +18,82 @@
 
 package org.apache.tajo.master.querymaster;
 
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.SystemClock;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.client.YarnClient;
-import org.apache.hadoop.yarn.client.YarnClientImpl;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.proto.YarnProtos;
-import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.service.Service;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
 import org.apache.tajo.engine.planner.global.GlobalOptimizer;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.planner.logical.LogicalNode;
-import org.apache.tajo.engine.planner.logical.LogicalRootNode;
-import org.apache.tajo.engine.planner.logical.NodeType;
-import org.apache.tajo.engine.planner.logical.ScanNode;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.master.*;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.rm.RMContainerAllocator;
-import org.apache.tajo.rpc.ProtoAsyncRpcServer;
-import org.apache.tajo.rpc.ProtoBlockingRpcClient;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.GlobalPlanner;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.event.QueryStartEvent;
+import org.apache.tajo.rpc.NullCallback;
 import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.worker.TajoWorker;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicBoolean;
 
-// TODO - when exception, send error status to QueryMasterManager
+// TODO - when exception, send error status to QueryJobManager
 public class QueryMaster extends CompositeService implements EventHandler {
   private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName());
-  private static PrimitiveProtos.BoolProto TRUE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
-  private static PrimitiveProtos.BoolProto FALSE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
   private static int QUERY_SESSION_TIMEOUT = 60 * 1000;  //60 sec
 
-  // AppMaster Common
-  private final long appSubmitTime;
   private Clock clock;
 
-  // For Query
-  private final QueryId queryId;
-  private QueryContext queryContext;
-  private Query query;
-  private TajoProtos.QueryState state = TajoProtos.QueryState.QUERY_NOT_ASSIGNED;
-  private String statusMessage;
-  private MasterPlan masterPlan;
-
-  private AsyncDispatcher dispatcher;
-  private RMContainerAllocator rmAllocator;
+  private TajoAsyncDispatcher dispatcher;
 
-  //service handler for QueryMasterManager, Worker
-  private QueryMasterService queryMasterService;
-  private QueryMasterClientService queryMasterClientService;
-
-  private TaskRunnerLauncher taskRunnerLauncher;
   private GlobalPlanner globalPlanner;
+
   private GlobalOptimizer globalOptimizer;
 
-  private boolean isCreateTableStmt;
+//  private boolean isCreateTableStmt;
   private StorageManager storageManager;
-  private Path outputPath;
+
   private QueryConf queryConf;
-  private ApplicationAttemptId appAttemptId;
-  private ApplicationId appId;
-  private ProtoBlockingRpcClient queryMasterManagerClient;
-  private QueryMasterManagerProtocol.QueryMasterManagerProtocolService.BlockingInterface queryMasterManagerService;
 
-  private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+  private Map<QueryId, QueryMasterTask> queryMasterTasks = new HashMap<QueryId, QueryMasterTask>();
 
-  private String queryMasterManagerAddress;
+  private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
 
-  private YarnRPC yarnRPC;
+  private AtomicBoolean queryMasterStop = new AtomicBoolean(false);
 
-  private YarnClient yarnClient;
+  private QueryMasterContext queryMasterContext;
 
-  private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
+  private QueryHeartbeatThread queryHeartbeatThread;
 
-  public QueryMaster(final QueryId queryId, final long appSubmitTime, String queryMasterManagerAddress) {
-    super(QueryMaster.class.getName());
+  private TajoWorker.WorkerContext workerContext;
 
-    this.queryId = queryId;
-    this.appSubmitTime = appSubmitTime;
-    this.appId = queryId.getApplicationId();
-    this.queryMasterManagerAddress = queryMasterManagerAddress;
-
-    LOG.info("Created Query Master for " + queryId);
+  public QueryMaster(TajoWorker.WorkerContext workerContext) {
+    super(QueryMaster.class.getName());
+    this.workerContext = workerContext;
   }
 
   public void init(Configuration conf) {
+    LOG.info("QueryMaster init");
     try {
       queryConf = new QueryConf(conf);
-      QUERY_SESSION_TIMEOUT = 60 * 1000;//queryConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
-      queryContext = new QueryContext(queryConf);
-      yarnRPC = YarnRPC.create(queryContext.getConf());
-      connectYarnClient();
+      queryConf.addResource(new Path(QueryConf.QUERY_MASTER_FILENAME));
 
-      LOG.info("Init QueryMasterManagerClient connection to:" + queryMasterManagerAddress);
-      InetSocketAddress addr = NetUtils.createSocketAddr(queryMasterManagerAddress);
-      queryMasterManagerClient = new ProtoBlockingRpcClient(QueryMasterManagerProtocol.class, addr);
-      queryMasterManagerService = queryMasterManagerClient.getStub();
+      QUERY_SESSION_TIMEOUT = 60 * 1000;//queryConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
+      queryMasterContext = new QueryMasterContext(queryConf);
 
       clock = new SystemClock();
 
-      this.dispatcher = new AsyncDispatcher();
+      this.dispatcher = new TajoAsyncDispatcher("querymaster_" + System.currentTimeMillis());
       addIfService(dispatcher);
 
       this.storageManager = new StorageManager(queryConf);
@@ -161,23 +101,8 @@ public class QueryMaster extends CompositeService implements EventHandler {
       globalPlanner = new GlobalPlanner(queryConf, storageManager, dispatcher.getEventHandler());
       globalOptimizer = new GlobalOptimizer();
 
-      queryMasterService = new QueryMasterService();
-      addIfService(queryMasterService);
-
-      queryMasterClientService = new QueryMasterClientService(queryContext);
-      addIfService(queryMasterClientService);
-
-      initStagingDir();
-
-      dispatcher.register(SubQueryEventType.class, new SubQueryEventDispatcher());
-      dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
-      dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
-      dispatcher.register(QueryFinishEvent.EventType.class, new QueryFinishEventHandler());
-      dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+      dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
 
-      clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
-
-      clientSessionTimeoutCheckThread.start();
     } catch (Throwable t) {
       LOG.error(t.getMessage(), t);
       throw new RuntimeException(t);
@@ -185,364 +110,88 @@ public class QueryMaster extends CompositeService implements EventHandler {
     super.init(conf);
   }
 
-  class ClientSessionTimeoutCheckThread extends Thread {
-    public void run() {
-      LOG.info("ClientSessionTimeoutCheckThread started");
-      while(true) {
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          break;
-        }
-        try {
-          long lastHeartbeat = queryContext.getLastClientHeartbeat();
-          long time = System.currentTimeMillis() - lastHeartbeat;
-          if(lastHeartbeat > 0 && time > QUERY_SESSION_TIMEOUT) {
-            LOG.warn("Query " + queryId + " stopped cause query sesstion timeout: " + time + " ms");
-            QueryMaster.this.stop();
-          }
-        } catch (Exception e) {
-          LOG.error(e.getMessage(), e);
-        }
-      }
-    }
-  }
-
-  class QueryHeartbeatThread extends Thread {
-    public QueryHeartbeatThread() {
-      super("QueryHeartbeatThread");
-    }
-
-    @Override
-    public void run() {
-      LOG.info("Start QueryMaster heartbeat thread");
-      while(queryMasterManagerClient.isConnected()) {
-        QueryMasterManagerProtocol.QueryHeartbeat queryHeartbeat =
-            QueryMasterManagerProtocol.QueryHeartbeat.newBuilder()
-                .setQueryMasterHost(queryMasterService.bindAddr.getHostName())
-                .setQueryMasterPort(queryMasterService.bindAddr.getPort())
-                .setQueryMasterClientPort(queryMasterClientService.getBindAddr().getPort())
-                .setState(state)
-                .setQueryId(queryId.getProto())
-                .build();
-
-        try {
-          QueryMasterManagerProtocol.QueryHeartbeatResponse response =
-              queryMasterManagerService.queryHeartbeat(null, queryHeartbeat);
-          if(response.getResponseCommand() != null) {
-            if("executeQuery".equals(response.getResponseCommand().getCommand())) {
-              appAttemptId = TajoIdUtils.toApplicationAttemptId(response.getResponseCommand().getParams(0));
-              startQuery(response.getResponseCommand().getParams(1),
-                  response.getResponseCommand().getParams(2));
-            }
-          }
-        } catch (Exception e) {
-          LOG.error(e.getMessage(), e);
-        }
-        try {
-          Thread.sleep(2000);
-        } catch (InterruptedException e) {
-          break;
-        }
-      }
-      LOG.info("QueryMaster heartbeat thread stopped");
-    }
-  }
-
-  // TODO blocking/nonblocking ???
-  class QueryMasterService extends AbstractService implements QueryMasterProtocol.QueryMasterProtocolService.Interface {
-    private ProtoAsyncRpcServer rpcServer;
-    private InetSocketAddress bindAddr;
-    private String addr;
-    private QueryHeartbeatThread queryHeartbeatThread;
-
-    public QueryMasterService() {
-      super(QueryMasterService.class.getName());
-
-      // Setup RPC server
-      try {
-        InetSocketAddress initIsa =
-                new InetSocketAddress(InetAddress.getLocalHost(), 0);
-        if (initIsa.getAddress() == null) {
-          throw new IllegalArgumentException("Failed resolve of " + initIsa);
-        }
-
-        this.rpcServer = new ProtoAsyncRpcServer(QueryMasterProtocol.class, this, initIsa);
-        this.rpcServer.start();
-
-        this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
-        this.addr = NetUtils.normalizeInetSocketAddress(this.bindAddr);
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-      }
-      queryConf.setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
-      LOG.info("QueryMasterService startup");
-    }
-
-    @Override
-    public void init(Configuration conf) {
-      super.init(conf);
-    }
-
-    @Override
-    public void start() {
-      try {
-        queryHeartbeatThread = new QueryHeartbeatThread();
-        queryHeartbeatThread.start();
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
-        // TODO - set query status failed and stop QueryMaster
-      }
-      super.start();
-    }
-
-    @Override
-    public void stop() {
-      if(rpcServer != null) {
-        rpcServer.shutdown();
-      }
-      if(queryHeartbeatThread != null) {
-        queryHeartbeatThread.interrupt();
-      }
-      if(yarnClient != null) {
-        yarnClient.stop();
-      }
-      if(clientSessionTimeoutCheckThread != null) {
-        clientSessionTimeoutCheckThread.interrupt();
-      }
-      super.stop();
-      LOG.info("QueryMasterService stopped");
-    }
-
-    @Override
-    public void getTask(RpcController controller, YarnProtos.ContainerIdProto request,
-                        RpcCallback<QueryMasterProtocol.QueryUnitRequestProto> done) {
-      queryContext.getEventHandler().handle(new TaskRequestEvent(new ContainerIdPBImpl(request), done));
-    }
-
-    @Override
-    public void statusUpdate(RpcController controller, QueryMasterProtocol.TaskStatusProto request,
-                             RpcCallback<PrimitiveProtos.BoolProto> done) {
-      QueryUnitAttemptId attemptId = new QueryUnitAttemptId(request.getId());
-      queryContext.getEventHandler().handle(new TaskAttemptStatusUpdateEvent(attemptId, request));
-      done.run(TRUE_PROTO);
-    }
-
-    @Override
-    public void ping(RpcController controller,
-                     TajoIdProtos.QueryUnitAttemptIdProto attemptIdProto,
-                     RpcCallback<PrimitiveProtos.BoolProto> done) {
-      // TODO - to be completed
-      QueryUnitAttemptId attemptId = new QueryUnitAttemptId(attemptIdProto);
-      done.run(TRUE_PROTO);
-    }
-
-    @Override
-    public void fatalError(RpcController controller, QueryMasterProtocol.TaskFatalErrorReport report,
-                           RpcCallback<PrimitiveProtos.BoolProto> done) {
-      queryContext.getEventHandler().handle(new TaskFatalErrorEvent(report));
-      done.run(TRUE_PROTO);
-    }
+  @Override
+  public void start() {
+    LOG.info("====>QueryMaster start");
 
-    @Override
-    public void done(RpcController controller, QueryMasterProtocol.TaskCompletionReport report,
-                     RpcCallback<PrimitiveProtos.BoolProto> done) {
-      queryContext.getEventHandler().handle(new TaskCompletionEvent(report));
-      done.run(TRUE_PROTO);
-    }
+    queryHeartbeatThread = new QueryHeartbeatThread();
+    queryHeartbeatThread.start();
 
-    @Override
-    public void executeQuery(RpcController controller, PrimitiveProtos.StringProto request,
-                             RpcCallback<PrimitiveProtos.BoolProto> done) {
-    }
-  }
+    clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
+    clientSessionTimeoutCheckThread.start();
 
-  public void start() {
     super.start();
   }
 
+  @Override
   public void stop() {
-    LOG.info("unregisterApplicationMaster");
-    if(rmAllocator != null) {
-      try {
-        FinalApplicationStatus status = FinalApplicationStatus.UNDEFINED;
-        if (query != null) {
-          TajoProtos.QueryState state = query.getState();
-          if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
-            status = FinalApplicationStatus.SUCCEEDED;
-          } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
-            status = FinalApplicationStatus.FAILED;
-          } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
-            status = FinalApplicationStatus.FAILED;
-          }
-        }
-        this.rmAllocator.unregisterApplicationMaster(status, "tajo query finished", null);
-      } catch (Exception e) {
-        LOG.error(e.getMessage(), e);
+    synchronized(queryMasterStop) {
+      if(queryMasterStop.get()) {
+         return;
       }
-    }
 
-    // TODO - release opened resource
-    if(this.queryMasterManagerClient != null) {
-      reportQueryStatus();
-
-      queryMasterManagerClient.close();
+      queryMasterStop.set(true);
+      queryMasterStop.notifyAll();
     }
 
-    try {
-      FileSystem.closeAll();
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
+    if(queryHeartbeatThread != null) {
+      queryHeartbeatThread.interrupt();
     }
 
+    if(clientSessionTimeoutCheckThread != null) {
+      clientSessionTimeoutCheckThread.interrupt();
+    }
     super.stop();
 
-    synchronized(queryId) {
-      queryId.notifyAll();
+    LOG.info("QueryMaster stop");
+    if(!queryMasterContext.getWorkerContext().isStandbyMode()) {
+      queryMasterContext.getWorkerContext().stopWorker(true);
     }
   }
 
-  private void reportQueryStatus() {
-    //send query status heartbeat
-    QueryMasterManagerProtocol.QueryHeartbeat.Builder queryHeartbeatBuilder =
-        QueryMasterManagerProtocol.QueryHeartbeat.newBuilder()
-        .setQueryMasterHost(queryMasterService.bindAddr.getHostName())
-        .setQueryMasterPort(queryMasterService.bindAddr.getPort())
-        .setQueryMasterClientPort(queryMasterClientService.getBindAddr().getPort())
-        .setState(state)
-        .setQueryId(queryId.getProto());
-
-    if(statusMessage != null) {
-      queryHeartbeatBuilder.setStatusMessage(statusMessage);
-    }
+  public void reportQueryStatusToQueryMaster(QueryId queryId, TajoProtos.QueryState state) {
+    LOG.info("Send QueryMaster Ready to QueryJobManager:" + queryId);
     try {
-      queryMasterManagerService.queryHeartbeat(null, queryHeartbeatBuilder.build());
+      TajoMasterProtocol.TajoHeartbeat.Builder queryHeartbeatBuilder = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+          .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+          .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+          .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+          .setState(state)
+          .setQueryId(queryId.getProto());
+
+      workerContext.getTajoMasterRpcClient().heartbeat(null, queryHeartbeatBuilder.build(), NullCallback.get());
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
     }
   }
 
-  private void connectYarnClient() {
-    this.yarnClient = new YarnClientImpl();
-    this.yarnClient.init(queryConf);
-    this.yarnClient.start();
-  }
-
   protected void addIfService(Object object) {
     if (object instanceof Service) {
       addService((Service) object);
     }
   }
 
-  public synchronized void startQuery(String queryStr, String planJSON) {
-    LOG.info("Query Start:" + queryStr);
-    LOG.info("Plan JSON:" + planJSON);
-    if(query != null) {
-      LOG.warn("Query already started");
-      return;
-    }
-
-    try {
-      LogicalRootNode logicalNodeRoot = (LogicalRootNode) CoreGsonHelper.fromJson(planJSON, LogicalNode.class);
-      LogicalNode[] scanNodes = PlannerUtil.findAllNodes(logicalNodeRoot, NodeType.SCAN);
-      if(scanNodes != null) {
-        for(LogicalNode eachScanNode: scanNodes) {
-          ScanNode scanNode = (ScanNode)eachScanNode;
-          tableDescMap.put(scanNode.getFromTable().getTableName(), scanNode.getFromTable().getTableDesc());
-        }
-      }
-      MasterPlan globalPlan = globalPlanner.build(queryId, logicalNodeRoot);
-      this.masterPlan = globalOptimizer.optimize(globalPlan);
-
-      taskRunnerLauncher = new TaskRunnerLauncherImpl(queryContext);
-      addIfService(taskRunnerLauncher);
-      dispatcher.register(TaskRunnerGroupEvent.EventType.class, taskRunnerLauncher);
-
-      ((TaskRunnerLauncherImpl)taskRunnerLauncher).init(queryConf);
-      ((TaskRunnerLauncherImpl)taskRunnerLauncher).start();
-
-      rmAllocator = new RMContainerAllocator(queryContext);
-      addIfService(rmAllocator);
-      dispatcher.register(ContainerAllocatorEventType.class, rmAllocator);
-
-      rmAllocator.init(queryConf);
-      rmAllocator.start();
-
-      //TODO - synchronized with executeQuery logic
-      query = new Query(queryContext, queryId, clock, appSubmitTime,
-              "", dispatcher.getEventHandler(), masterPlan, storageManager);
-      dispatcher.register(QueryEventType.class, query);
-
-      dispatcher.getEventHandler().handle(new QueryEvent(queryId,
-          QueryEventType.INIT));
-      dispatcher.getEventHandler().handle(new QueryEvent(queryId,
-          QueryEventType.START));
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-      //send FAIL query status
-      this.statusMessage = StringUtils.stringifyException(e);
-      this.state = TajoProtos.QueryState.QUERY_FAILED;
-    }
-  }
-
   @Override
   public void handle(Event event) {
     dispatcher.getEventHandler().handle(event);
   }
 
-  public EventHandler getEventHandler() {
-    return dispatcher.getEventHandler();
-  }
-
-  private class SubQueryEventDispatcher implements EventHandler<SubQueryEvent> {
-    public void handle(SubQueryEvent event) {
-      SubQueryId id = event.getSubQueryId();
-      query.getSubQuery(id).handle(event);
-    }
+  public Query getQuery(QueryId queryId) {
+    return queryMasterTasks.get(queryId).getQuery();
   }
 
-  private class TaskEventDispatcher
-      implements EventHandler<TaskEvent> {
-    public void handle(TaskEvent event) {
-      QueryUnitId taskId = event.getTaskId();
-      QueryUnit task = query.getSubQuery(taskId.getSubQueryId()).
-          getQueryUnit(taskId);
-      task.handle(event);
-    }
-  }
-
-  private class TaskAttemptEventDispatcher
-      implements EventHandler<TaskAttemptEvent> {
-    public void handle(TaskAttemptEvent event) {
-      QueryUnitAttemptId attemptId = event.getTaskAttemptId();
-      SubQuery subQuery = query.getSubQuery(attemptId.getSubQueryId());
-      QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
-      QueryUnitAttempt attempt = task.getAttempt(attemptId);
-      attempt.handle(event);
-    }
-  }
-
-  private class TaskSchedulerDispatcher
-      implements EventHandler<TaskSchedulerEvent> {
-    public void handle(TaskSchedulerEvent event) {
-      SubQuery subQuery = query.getSubQuery(event.getSubQueryId());
-      subQuery.getTaskScheduler().handle(event);
-    }
+  public QueryMasterTask getQueryMasterTask(QueryId queryId) {
+    return queryMasterTasks.get(queryId);
   }
 
-  public QueryContext getContext() {
-    return this.queryContext;
+  public QueryMasterContext getContext() {
+    return this.queryMasterContext;
   }
 
-  public class QueryContext {
+  public class QueryMasterContext {
     private QueryConf conf;
-    public Map<ContainerId, ContainerProxy> containers = new ConcurrentHashMap<ContainerId, ContainerProxy>();
-    int minCapability;
-    int maxCapability;
-    int numCluster;
-    AtomicLong lastClientHeartbeat = new AtomicLong(-1);
 
-    public QueryContext(QueryConf conf) {
+    public QueryMasterContext(QueryConf conf) {
       this.conf = conf;
     }
 
@@ -550,15 +199,7 @@ public class QueryMaster extends CompositeService implements EventHandler {
       return conf;
     }
 
-    public InetSocketAddress getQueryMasterServiceAddress() {
-      return queryMasterService.bindAddr;
-    }
-
-    public QueryMasterClientService getQueryMasterClientService() {
-      return queryMasterClientService;
-    }
-
-    public AsyncDispatcher getDispatcher() {
+    public TajoAsyncDispatcher getDispatcher() {
       return dispatcher;
     }
 
@@ -566,251 +207,133 @@ public class QueryMaster extends CompositeService implements EventHandler {
       return clock;
     }
 
-    public Query getQuery() {
-      return query;
+    public StorageManager getStorageManager() {
+      return storageManager;
     }
 
-    public SubQuery getSubQuery(SubQueryId subQueryId) {
-      return query.getSubQuery(subQueryId);
+    public QueryMaster getQueryMaster() {
+      return QueryMaster.this;
     }
 
-    public QueryId getQueryId() {
-      return queryId;
+    public GlobalPlanner getGlobalPlanner() {
+      return globalPlanner;
     }
-
-    public ApplicationId getApplicationId() {
-      return appId;
+    public GlobalOptimizer getGlobalOptimizer() {
+      return globalOptimizer;
     }
 
-    public ApplicationAttemptId getApplicationAttemptId() {
-      return appAttemptId;
+    public TajoWorker.WorkerContext getWorkerContext() {
+      return workerContext;
     }
 
     public EventHandler getEventHandler() {
       return dispatcher.getEventHandler();
     }
 
-    public void addContainer(ContainerId cId, ContainerProxy container) {
-      containers.put(cId, container);
-    }
-
-    public void removeContainer(ContainerId cId) {
-      containers.remove(cId);
-    }
-
-    public boolean containsContainer(ContainerId cId) {
-      return containers.containsKey(cId);
-    }
-
-    public ContainerProxy getContainer(ContainerId cId) {
-      return containers.get(cId);
-    }
-
-    public Map<ContainerId, ContainerProxy> getContainers() {
-      return containers;
-    }
-
-    public int getNumClusterNode() {
-      return numCluster;
-    }
-
-    public void setNumClusterNodes(int num) {
-      numCluster = num;
-    }
-
-//    public CatalogService getCatalog() {
-//      return catalog;
-//    }
-
-    public Map<String, TableDesc> getTableDescMap() {
-      return tableDescMap;
-    }
-
-    public Path getOutputPath() {
-      return outputPath;
-    }
-
-    public void setMaxContainerCapability(int capability) {
-      this.maxCapability = capability;
-    }
-
-    public int getMaxContainerCapability() {
-      return this.maxCapability;
-    }
-
-    public void setMinContainerCapability(int capability) {
-      this.minCapability = capability;
-    }
-
-    public int getMinContainerCapability() {
-      return this.minCapability;
-    }
-
-    public boolean isCreateTableQuery() {
-      return isCreateTableStmt;
-    }
-
-    public float getProgress() {
-      if(query != null) {
-        return query.getProgress();
-      } else {
-        return 0;
+    public void stopQuery(QueryId queryId) {
+      QueryMasterTask queryMasterTask;
+      synchronized(queryMasterTasks) {
+        queryMasterTask = queryMasterTasks.remove(queryId);
       }
-    }
-
-    public long getStartTime() {
-      if(query != null) {
-        return query.getStartTime();
+      if(queryMasterTask != null) {
+        try {
+          queryMasterTask.stop();
+        } catch (Exception e) {
+          LOG.error(e.getMessage(), e);
+        }
       } else {
-        return -1;
+        LOG.warn("No query info:" + queryId);
       }
-    }
-
-    public long getFinishTime() {
-      if(query != null) {
-        return query.getFinishTime();
-      } else {
-        return -1;
+      if(!workerContext.isStandbyMode()) {
+        stop();
       }
     }
-
-    public StorageManager getStorageManager() {
-      return storageManager;
-    }
-
-    public QueryMaster getQueryMaster() {
-      return QueryMaster.this;
-    }
-
-    public YarnRPC getYarnRPC() {
-      return yarnRPC;
-    }
-
-    public void setState(TajoProtos.QueryState state) {
-      QueryMaster.this.state = state;
-    }
-
-    public TajoProtos.QueryState getState() {
-      return state;
-    }
-
-    public void touchSessionTime() {
-      this.lastClientHeartbeat.set(System.currentTimeMillis());
-    }
-
-    public long getLastClientHeartbeat() {
-      return this.lastClientHeartbeat.get();
-    }
   }
 
-  private class QueryFinishEventHandler implements EventHandler<QueryFinishEvent> {
+  private class QueryStartEventHandler implements EventHandler<QueryStartEvent> {
     @Override
-    public void handle(QueryFinishEvent event) {
-      LOG.info("Query end notification started for QueryId : " + query.getId() + "," + query.getState());
-
-      //QueryMaster must be lived until client fetching all query result data.
-      try {
-        // Stop all services
-        // This will also send the final report to the ResourceManager
-        //LOG.info("Calling stop for all the services");
-//        stop();
-      } catch (Throwable t) {
-        LOG.warn("Graceful stop failed ", t);
+    public void handle(QueryStartEvent event) {
+      LOG.info("====>Start QueryStartEventHandler:" + event.getQueryId());
+      //To change body of implemented methods use File | Settings | File Templates.
+      QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
+          event.getQueryId(), event.getLogicalPlanJson());
+
+      queryMasterTask.init(queryConf);
+      queryMasterTask.start();
+      synchronized(queryMasterTasks) {
+        queryMasterTasks.put(event.getQueryId(), queryMasterTask);
       }
-
-      //Bring the process down by force.
-      //Not needed after HADOOP-7140
-      //LOG.info("Exiting QueryMaster..GoodBye!");
     }
   }
 
-  // query submission directory is private!
-  final public static FsPermission USER_DIR_PERMISSION =
-      FsPermission.createImmutable((short) 0700); // rwx--------
-
-  /**
-   * It initializes the final output and staging directory and sets
-   * them to variables.
-   */
-  private void initStagingDir() throws IOException {
-    QueryConf conf = getContext().getConf();
-
-    String realUser;
-    String currentUser;
-    UserGroupInformation ugi;
-    ugi = UserGroupInformation.getLoginUser();
-    realUser = ugi.getShortUserName();
-    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
-
-    String givenOutputTableName = conf.getOutputTable();
-    Path stagingDir;
-
-    // If final output directory is not given by an user,
-    // we use the query id as a output directory.
-    if (givenOutputTableName.equals("")) {
-      this.isCreateTableStmt = false;
-      FileSystem defaultFS = FileSystem.get(conf);
-
-      Path homeDirectory = defaultFS.getHomeDirectory();
-      if (!defaultFS.exists(homeDirectory)) {
-        defaultFS.mkdirs(homeDirectory, new FsPermission(USER_DIR_PERMISSION));
-      }
-
-      Path userQueryDir = new Path(homeDirectory, TajoConstants.USER_QUERYDIR_PREFIX);
-
-      if (defaultFS.exists(userQueryDir)) {
-        FileStatus fsStatus = defaultFS.getFileStatus(userQueryDir);
-        String owner = fsStatus.getOwner();
+  class QueryHeartbeatThread extends Thread {
+    public QueryHeartbeatThread() {
+      super("QueryHeartbeatThread");
+    }
 
-        if (!(owner.equals(currentUser) || owner.equals(realUser))) {
-          throw new IOException("The ownership on the user's query " +
-              "directory " + userQueryDir + " is not as expected. " +
-              "It is owned by " + owner + ". The directory must " +
-              "be owned by the submitter " + currentUser + " or " +
-              "by " + realUser);
+    @Override
+    public void run() {
+      LOG.info("Start QueryMaster heartbeat thread");
+      while(!queryMasterStop.get()) {
+        //TODO report all query status
+        List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+        synchronized(queryMasterTasks) {
+          tempTasks.addAll(queryMasterTasks.values());
         }
+        synchronized(queryMasterTasks) {
+          for(QueryMasterTask eachTask: tempTasks) {
+            TajoMasterProtocol.TajoHeartbeat queryHeartbeat = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+                .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+                .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+                .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+                .setState(eachTask.getState())
+                .setQueryId(eachTask.getQueryId().getProto())
+                .build();
 
-        if (!fsStatus.getPermission().equals(USER_DIR_PERMISSION)) {
-          LOG.info("Permissions on staging directory " + userQueryDir + " are " +
-              "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
-              "to correct value " + USER_DIR_PERMISSION);
-          defaultFS.setPermission(userQueryDir, new FsPermission(USER_DIR_PERMISSION));
+            workerContext.getTajoMasterRpcClient().heartbeat(null, queryHeartbeat, NullCallback.get());
+          }
+        }
+        synchronized(queryMasterStop) {
+          try {
+            queryMasterStop.wait(2000);
+          } catch (InterruptedException e) {
+            break;
+          }
         }
-      } else {
-        defaultFS.mkdirs(userQueryDir,
-            new FsPermission(USER_DIR_PERMISSION));
-      }
-
-      stagingDir = StorageUtil.concatPath(userQueryDir, queryId.toString());
-
-      if (defaultFS.exists(stagingDir)) {
-        throw new IOException("The staging directory " + stagingDir
-            + "already exists. The directory must be unique to each query");
-      } else {
-        defaultFS.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
       }
+      LOG.info("QueryMaster heartbeat thread stopped");
+    }
+  }
 
-      // Set the query id to the output table name
-      conf.setOutputTable(queryId.toString());
 
-    } else {
-      this.isCreateTableStmt = true;
-      Path warehouseDir = new Path(conf.getVar(TajoConf.ConfVars.ROOT_DIR),
-          TajoConstants.WAREHOUSE_DIR);
-      stagingDir = new Path(warehouseDir, conf.getOutputTable());
+  class ClientSessionTimeoutCheckThread extends Thread {
+    public void run() {
+      LOG.info("ClientSessionTimeoutCheckThread started");
+      while(true) {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+          break;
+        }
+        List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+        synchronized(queryMasterTasks) {
+          tempTasks.addAll(queryMasterTasks.values());
+        }
 
-      FileSystem fs = warehouseDir.getFileSystem(conf);
-      if (fs.exists(stagingDir)) {
-        throw new IOException("The staging directory " + stagingDir
-            + " already exists. The directory must be unique to each query");
-      } else {
-        // TODO - should have appropriate permission
-        fs.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
+        for(QueryMasterTask eachTask: tempTasks) {
+          try {
+            long lastHeartbeat = eachTask.getLastClientHeartbeat();
+            long time = System.currentTimeMillis() - lastHeartbeat;
+            if(lastHeartbeat > 0 && time > QUERY_SESSION_TIMEOUT) {
+              LOG.warn("Query " + eachTask.getQueryId() + " stopped cause query sesstion timeout: " + time + " ms");
+              eachTask.expiredSessionTimeout();
+            }
+          } catch (Exception e) {
+            LOG.error(eachTask.getQueryId() + ":" + e.getMessage(), e);
+          }
+        }
       }
     }
-
-    conf.setOutputPath(stagingDir);
-    outputPath = stagingDir;
-    LOG.info("Initialized Query Staging Dir: " + outputPath);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
deleted file mode 100644
index 74298e5..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.proto.YarnProtos;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.ipc.ClientProtos;
-import org.apache.tajo.ipc.QueryMasterClientProtocol;
-import org.apache.tajo.rpc.ProtoBlockingRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-
-public class QueryMasterClientService extends AbstractService {
-  private static final Log LOG = LogFactory.getLog(QueryMasterClientService.class);
-  private final PrimitiveProtos.BoolProto BOOL_TRUE =
-          PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
-
-  private ProtoBlockingRpcServer rpcServer;
-  private InetSocketAddress bindAddr;
-  private String addr;
-  private QueryMaster.QueryContext queryContext;
-  private QueryMasterClientProtocolServiceHandler serviceHandler;
-
-  public QueryMasterClientService(QueryMaster.QueryContext queryContext) {
-    super(QueryMasterClientService.class.getName());
-
-    this.queryContext = queryContext;
-    this.serviceHandler = new QueryMasterClientProtocolServiceHandler();
-
-    // init RPC Server in constructor cause Heartbeat Thread use bindAddr
-    // Setup RPC server
-    try {
-      // TODO initial port num is value of config and find unused port with sequence
-      InetSocketAddress initIsa = new InetSocketAddress(InetAddress.getLocalHost(), 0);
-      if (initIsa.getAddress() == null) {
-        throw new IllegalArgumentException("Failed resolve of " + initIsa);
-      }
-
-      // TODO blocking/non-blocking??
-      this.rpcServer = new ProtoBlockingRpcServer(QueryMasterClientProtocol.class, serviceHandler, initIsa);
-      this.rpcServer.start();
-
-      this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
-      this.addr = NetUtils.normalizeInetSocketAddress(bindAddr);
-    } catch (Exception e) {
-      LOG.error(e.getMessage(), e);
-    }
-    // Get the master address
-    LOG.info(QueryMasterClientService.class.getSimpleName() + " (" + queryContext.getQueryId() + ") listens on "
-        + addr);
-  }
-
-  @Override
-  public void init(Configuration conf) {
-    super.init(conf);
-  }
-
-  @Override
-  public void start() {
-    super.start();
-  }
-
-  @Override
-  public void stop() {
-    if(rpcServer != null) {
-      rpcServer.shutdown();
-    }
-    LOG.info("QueryMasterClientService stopped");
-    super.stop();
-  }
-
-  public InetSocketAddress getBindAddr() {
-    return bindAddr;
-  }
-
-
-  public class QueryMasterClientProtocolServiceHandler
-          implements QueryMasterClientProtocol.QueryMasterClientProtocolService.BlockingInterface {
-    @Override
-    public PrimitiveProtos.BoolProto updateSessionVariables(
-            RpcController controller,
-            ClientProtos.UpdateSessionVariableRequest request) throws ServiceException {
-      return null;
-    }
-
-    @Override
-    public ClientProtos.GetQueryResultResponse getQueryResult(
-            RpcController controller,
-            ClientProtos.GetQueryResultRequest request) throws ServiceException {
-      QueryId queryId = new QueryId(request.getQueryId());
-      Query query = queryContext.getQuery();
-
-      ClientProtos.GetQueryResultResponse.Builder builder = ClientProtos.GetQueryResultResponse.newBuilder();
-
-      if(query == null) {
-        builder.setErrorMessage("No Query for " + queryId);
-      } else {
-        switch (query.getState()) {
-          case QUERY_SUCCEEDED:
-            builder.setTableDesc((CatalogProtos.TableDescProto)query.getResultDesc().getProto());
-            break;
-          case QUERY_FAILED:
-          case QUERY_ERROR:
-            builder.setErrorMessage("Query " + queryId + " is failed");
-          default:
-            builder.setErrorMessage("Query " + queryId + " is still running");
-        }
-      }
-      return builder.build();
-    }
-
-    @Override
-    public ClientProtos.GetQueryStatusResponse getQueryStatus(
-            RpcController controller,
-            ClientProtos.GetQueryStatusRequest request) throws ServiceException {
-      ClientProtos.GetQueryStatusResponse.Builder builder
-              = ClientProtos.GetQueryStatusResponse.newBuilder();
-      QueryId queryId = new QueryId(request.getQueryId());
-      builder.setQueryId(request.getQueryId());
-
-      if (queryId.equals(TajoIdUtils.NullQueryId)) {
-        builder.setResultCode(ClientProtos.ResultCode.OK);
-        builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
-      } else {
-        Query query = queryContext.getQuery();
-        builder.setResultCode(ClientProtos.ResultCode.OK);
-        builder.setQueryMasterHost(queryContext.getQueryMasterClientService().getBindAddr().getHostName());
-        builder.setQueryMasterPort(queryContext.getQueryMasterClientService().getBindAddr().getPort());
-
-
-        queryContext.touchSessionTime();
-        if (query != null) {
-          builder.setState(query.getState());
-          builder.setProgress(query.getProgress());
-          builder.setSubmitTime(query.getAppSubmitTime());
-          builder.setInitTime(query.getInitializationTime());
-          builder.setHasResult(!query.isCreateTableStmt());
-          if (query.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
-            builder.setFinishTime(query.getFinishTime());
-          } else {
-            builder.setFinishTime(System.currentTimeMillis());
-          }
-        } else {
-          builder.setState(queryContext.getState());
-        }
-      }
-
-      return builder.build();
-    }
-
-    @Override
-    public PrimitiveProtos.BoolProto killQuery(
-            RpcController controller,
-            YarnProtos.ApplicationAttemptIdProto request) throws ServiceException {
-      LOG.info("Stop QueryMaster:" + queryContext.getQueryId());
-      Thread t = new Thread() {
-        public void run() {
-          try {
-            Thread.sleep(1000);   //wait tile return to rpc response
-          } catch (InterruptedException e) {
-          }
-          queryContext.getQueryMaster().stop();
-        }
-      };
-      t.start();
-      return BOOL_TRUE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
deleted file mode 100644
index 35d7201..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.Clock;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.client.YarnClient;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.service.CompositeService;
-import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.engine.planner.logical.LogicalRootNode;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol.QueryHeartbeatResponse;
-import org.apache.tajo.master.ContainerProxy;
-import org.apache.tajo.master.TajoMaster;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-// TODO - check QueryMaster status and if QueryMaster failed, release resource
-public class QueryMasterManager extends CompositeService {
-  private static final Log LOG = LogFactory.getLog(QueryMasterManager.class.getName());
-
-  // Master Context
-  private final TajoMaster.MasterContext masterContext;
-
-  // AppMaster Common
-  private final Clock clock;
-  private final long appSubmitTime;
-  private final ApplicationId appId;
-  private ApplicationAttemptId appAttemptId;
-
-  protected YarnClient yarnClient;
-
-  // For Query
-  private final QueryId queryId;
-
-  private AsyncDispatcher dispatcher;
-  private YarnRPC rpc;
-
-  private TajoProtos.QueryState state;
-  private float progress;
-  private long finishTime;
-  private TableDesc resultDesc;
-  private String queryMasterHost;
-  private int queryMasterPort;
-  private int queryMasterClientPort;
-
-  private LogicalRootNode plan;
-
-  private AtomicBoolean querySubmitted = new AtomicBoolean(false);
-
-  private AtomicBoolean queryMasterStopped = new AtomicBoolean(true);
-
-  private boolean stopCheckThreadStarted = false;
-
-  private String query;
-
-  public QueryMasterManager(final TajoMaster.MasterContext masterContext,
-                     final YarnClient yarnClient,
-                     final QueryId queryId,
-                     final String query,
-                     final LogicalRootNode plan,
-                     final ApplicationId appId,
-                     final Clock clock, long appSubmitTime) {
-    super(QueryMasterManager.class.getName());
-    this.masterContext = masterContext;
-    this.yarnClient = yarnClient;
-
-    this.appId = appId;
-    this.clock = clock;
-    this.appSubmitTime = appSubmitTime;
-    this.queryId = queryId;
-    this.plan = plan;
-    this.query = query;
-    LOG.info("Created Query Master Manager for AppId=" + appId + ", QueryID=" + queryId);
-  }
-
-  @Override
-  public void init(Configuration conf) {
-    super.init(conf);
-
-    state = TajoProtos.QueryState.QUERY_MASTER_INIT;
-  }
-
-  public TajoProtos.QueryState getState() {
-    return state;
-  }
-
-  @Override
-  public void start() {
-    try {
-      appAttemptId = allocateAndLaunchQueryMaster();
-    } catch (YarnRemoteException e) {
-      LOG.error(e.getMessage(), e);
-    }
-    super.start();
-  }
-
-  @Override
-  public void stop() {
-    while(true) {
-      if(queryMasterStopped.get()) {
-        break;
-      }
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-      }
-    }
-    LOG.info("QueryMasterManager for " + queryId + " stopped");
-    super.stop();
-  }
-
-  public float getProgress() {
-    return progress;
-  }
-
-  public long getAppSubmitTime() {
-    return appSubmitTime;
-  }
-
-  public long getFinishTime() {
-    return finishTime;
-  }
-
-  public TableDesc getResultDesc() {
-    return resultDesc;
-  }
-
-  public String getQueryMasterHost() {
-    return queryMasterHost;
-  }
-
-  public int getQueryMasterPort() {
-    return queryMasterPort;
-  }
-
-  public int getQueryMasterClientPort() {
-    return queryMasterClientPort;
-  }
-
-  public synchronized QueryHeartbeatResponse.ResponseCommand queryHeartbeat(QueryMasterManagerProtocol.QueryHeartbeat queryHeartbeat) {
-    this.queryMasterHost = queryHeartbeat.getQueryMasterHost();
-    this.queryMasterPort = queryHeartbeat.getQueryMasterPort();
-    this.queryMasterClientPort = queryHeartbeat.getQueryMasterClientPort();
-    this.state = queryHeartbeat.getState();
-    if(state == TajoProtos.QueryState.QUERY_FAILED) {
-      //TODO needed QueryMaster's detail status(failed before or after launching worker)
-      queryMasterStopped.set(true);
-      if(queryHeartbeat.getStatusMessage() != null) {
-        LOG.warn(queryId + " failed, " + queryHeartbeat.getStatusMessage());
-      }
-    }
-
-    if(!stopCheckThreadStarted && !queryMasterStopped.get() && isFinishState(this.state)) {
-      stopCheckThreadStarted = true;
-      startCheckingQueryMasterStop();
-    }
-    if(appAttemptId != null && !querySubmitted.get()) {
-      LOG.info("submitQuery to QueryMaster(" + queryMasterHost + ":" + queryMasterPort + ")");
-      queryMasterStopped.set(false);
-      querySubmitted.set(true);
-      List<String> params = new ArrayList<String>(3);
-      params.add(appAttemptId.toString());
-      params.add(query);
-      params.add(plan.toJson());
-      return QueryHeartbeatResponse.ResponseCommand.newBuilder()
-          .setCommand("executeQuery")
-          .addAllParams(params)
-          .build();
-    } else {
-      return null;
-    }
-  }
-
-  private boolean isFinishState(TajoProtos.QueryState state) {
-    return state == TajoProtos.QueryState.QUERY_FAILED ||
-        state == TajoProtos.QueryState.QUERY_KILLED ||
-        state == TajoProtos.QueryState.QUERY_SUCCEEDED;
-  }
-
-  private void startCheckingQueryMasterStop() {
-    Thread t = new Thread() {
-      public void run() {
-        try {
-          ApplicationReport report = monitorApplication(appId,
-              EnumSet.of(
-                  YarnApplicationState.FINISHED,
-                  YarnApplicationState.KILLED,
-                  YarnApplicationState.FAILED));
-          queryMasterStopped.set(true);
-          LOG.info("QueryMaster (" + queryId + ") stopped");
-        } catch (YarnRemoteException e) {
-          LOG.error(e.getMessage(), e);
-        }
-      }
-    };
-
-    t.start();
-  }
-
-  private ApplicationAttemptId allocateAndLaunchQueryMaster() throws YarnRemoteException {
-    LOG.info("Allocate and launch QueryMaster:" + yarnClient);
-    ApplicationSubmissionContext appContext = Records.newRecord(ApplicationSubmissionContext.class);
-
-    // set the application id
-    appContext.setApplicationId(appId);
-    // set the application name
-    appContext.setApplicationName("Tajo");
-
-    Priority pri = Records.newRecord(Priority.class);
-    pri.setPriority(5);
-    appContext.setPriority(pri);
-
-    // Set the queue to which this application is to be submitted in the RM
-    appContext.setQueue("default");
-
-    ContainerLaunchContext commonContainerLaunchContext =
-            ContainerProxy.createCommonContainerLaunchContext(masterContext.getConf(), queryId.toString(), true);
-
-    // Setup environment by cloning from common env.
-    Map<String, String> env = commonContainerLaunchContext.getEnvironment();
-    Map<String, String> myEnv = new HashMap<String, String>(env.size());
-    myEnv.putAll(env);
-
-    ////////////////////////////////////////////////////////////////////////////
-    // Set the local resources
-    ////////////////////////////////////////////////////////////////////////////
-    // Set the necessary command to execute the application master
-    Vector<CharSequence> vargs = new Vector<CharSequence>(30);
-
-    // Set java executable command
-    //LOG.info("Setting up app master command");
-    vargs.add("${JAVA_HOME}" + "/bin/java");
-    // Set Xmx based on am memory size
-    vargs.add("-Xmx2000m");
-    // Set Remote Debugging
-    //if (!context.getQuery().getSubQuery(event.getSubQueryId()).isLeafQuery()) {
-    //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
-    //}
-    // Set class name
-    vargs.add(QueryMasterRunner.class.getCanonicalName());
-    vargs.add(queryId.toString()); // queryId
-    vargs.add(String.valueOf(appSubmitTime));
-    vargs.add(masterContext.getQueryMasterManagerService().getBindAddress().getHostName() + ":" +
-            masterContext.getQueryMasterManagerService().getBindAddress().getPort());
-
-    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
-    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
-
-    // Get final commmand
-    StringBuilder command = new StringBuilder();
-    for (CharSequence str : vargs) {
-      command.append(str).append(" ");
-    }
-
-    LOG.info("Completed setting up QueryMasterRunner command " + command.toString());
-    List<String> commands = new ArrayList<String>();
-    commands.add(command.toString());
-
-    final Resource resource = Records.newRecord(Resource.class);
-    // TODO - get default value from conf
-    resource.setMemory(2000);
-    resource.setVirtualCores(1);
-
-    Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
-
-    ContainerLaunchContext masterContainerContext = BuilderUtils.newContainerLaunchContext(
-            null, commonContainerLaunchContext.getUser(),
-            resource, commonContainerLaunchContext.getLocalResources(), myEnv, commands,
-            myServiceData, null, new HashMap<ApplicationAccessType, String>(2));
-
-    appContext.setAMContainerSpec(masterContainerContext);
-
-    LOG.info("Submitting QueryMaster to ResourceManager");
-    yarnClient.submitApplication(appContext);
-
-    ApplicationReport appReport = monitorApplication(appId, EnumSet.of(YarnApplicationState.ACCEPTED));
-    ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
-
-    LOG.info("Launching QueryMaster with id: " + attemptId);
-
-    state = TajoProtos.QueryState.QUERY_MASTER_LAUNCHED;
-
-    return attemptId;
-  }
-
-  private ApplicationReport monitorApplication(ApplicationId appId,
-                                               Set<YarnApplicationState> finalState) throws YarnRemoteException {
-
-    long sleepTime = 100;
-    int count = 1;
-    while (true) {
-      // Get application report for the appId we are interested in
-      ApplicationReport report = yarnClient.getApplicationReport(appId);
-
-      LOG.info("Got application report from ASM for" + ", appId="
-              + appId.getId() + ", appAttemptId="
-              + report.getCurrentApplicationAttemptId() + ", clientToken="
-              + report.getClientToken() + ", appDiagnostics="
-              + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
-              + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
-              + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
-              + ", yarnAppState=" + report.getYarnApplicationState().toString()
-              + ", distributedFinalState="
-              + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
-              + report.getTrackingUrl() + ", appUser=" + report.getUser());
-
-      YarnApplicationState state = report.getYarnApplicationState();
-      if (finalState.contains(state)) {
-        return report;
-      }
-      try {
-        Thread.sleep(sleepTime);
-        sleepTime = count * 100;
-        if(count < 10) {
-          count++;
-        }
-      } catch (InterruptedException e) {
-        //LOG.debug("Thread sleep in monitoring loop interrupted");
-      }
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
deleted file mode 100644
index 65f237c..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol.QueryHeartbeat;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol.QueryHeartbeatResponse;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.rpc.ProtoBlockingRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
-import org.apache.tajo.util.NetUtils;
-
-import java.net.InetSocketAddress;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-
-public class QueryMasterManagerService extends AbstractService {
-  private final static Log LOG = LogFactory.getLog(QueryMasterManagerService.class);
-
-  private final TajoMaster.MasterContext context;
-  private final TajoConf conf;
-  private final QueryMasterManagerProtocolServiceHandler masterHandler;
-  private ProtoBlockingRpcServer server;
-  private InetSocketAddress bindAddress;
-
-  private final BoolProto BOOL_TRUE = BoolProto.newBuilder().setValue(true).build();
-  private final BoolProto BOOL_FALSE = BoolProto.newBuilder().setValue(false).build();
-
-  public QueryMasterManagerService(TajoMaster.MasterContext context) {
-    super(QueryMasterManagerService.class.getName());
-    this.context = context;
-    this.conf = context.getConf();
-    this.masterHandler = new QueryMasterManagerProtocolServiceHandler();
-  }
-
-  @Override
-  public void start() {
-    // TODO resolve hostname
-    String confMasterServiceAddr = conf.getVar(ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS);
-    InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr);
-    try {
-      server = new ProtoBlockingRpcServer(QueryMasterManagerProtocol.class, masterHandler, initIsa);
-    } catch (Exception e) {
-      LOG.error(e);
-    }
-    server.start();
-    bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
-    this.conf.setVar(ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress));
-    LOG.info("QueryMasterManagerService startup");
-    super.start();
-  }
-
-  @Override
-  public void stop() {
-    if(server != null) {
-      server.shutdown();
-      server = null;
-    }
-    LOG.info("QueryMasterManagerService shutdown");
-    super.stop();
-  }
-
-  public InetSocketAddress getBindAddress() {
-    return bindAddress;
-  }
-
-  public class QueryMasterManagerProtocolServiceHandler implements QueryMasterManagerProtocol.QueryMasterManagerProtocolService.BlockingInterface {
-    @Override
-    public QueryHeartbeatResponse queryHeartbeat(RpcController controller, QueryHeartbeat request) throws ServiceException {
-      // TODO - separate QueryMasterManagerProtocol, ClientServiceProtocol
-      QueryId queryId = new QueryId(request.getQueryId());
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Received QueryHeartbeat:" + queryId + "," + request);
-      }
-      QueryMasterManager queryMasterManager = context.getQuery(queryId);
-      if (queryMasterManager == null) {
-        LOG.warn("No query:" + queryId);
-        return QueryHeartbeatResponse.newBuilder().setHeartbeatResult(BOOL_FALSE).build();
-      }
-
-      QueryHeartbeatResponse.ResponseCommand command = queryMasterManager.queryHeartbeat(request);
-
-      //ApplicationAttemptId attemptId = queryMasterManager.getAppAttemptId();
-      //String attemptIdStr = attemptId == null ? null : attemptId.toString();
-      QueryHeartbeatResponse.Builder builder = QueryHeartbeatResponse.newBuilder();
-      builder.setHeartbeatResult(BOOL_TRUE);
-      if(command != null) {
-        builder.setResponseCommand(command);
-      }
-      return builder.build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
index daab9fd..5a79464 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
@@ -39,13 +39,11 @@ public class QueryMasterRunner extends AbstractService {
   private QueryConf queryConf;
   private QueryMaster queryMaster;
   private QueryId queryId;
-  private long appSubmitTime;
   private String queryMasterManagerAddress;
 
-  public QueryMasterRunner(QueryId queryId, long appSubmitTime, String queryMasterManagerAddress) {
+  public QueryMasterRunner(QueryId queryId, String queryMasterManagerAddress) {
     super(QueryMasterRunner.class.getName());
     this.queryId = queryId;
-    this.appSubmitTime = appSubmitTime;
     this.queryMasterManagerAddress = queryMasterManagerAddress;
   }
 
@@ -72,7 +70,7 @@ public class QueryMasterRunner extends AbstractService {
   @Override
   public void start() {
     //create QueryMaster
-    QueryMaster query = new QueryMaster(queryId, appSubmitTime, queryMasterManagerAddress);
+    QueryMaster query = new QueryMaster(null);
 
     query.init(queryConf);
     query.start();
@@ -90,13 +88,12 @@ public class QueryMasterRunner extends AbstractService {
 
     UserGroupInformation.setConfiguration(conf);
 
-    final QueryId queryId = TajoIdUtils.createQueryId(args[0]);
-    final long appSubmitTime = Long.parseLong(args[1]);
-    final String queryMasterManagerAddr = args[2];
+    final QueryId queryId = TajoIdUtils.parseQueryId(args[0]);
+    final String queryMasterManagerAddr = args[1];
 
     LOG.info("Received QueryId:" + queryId);
 
-    QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, appSubmitTime, queryMasterManagerAddr);
+    QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, queryMasterManagerAddr);
     queryMasterRunner.init(conf);
     queryMasterRunner.start();
 


[8/8] git commit: TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)


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

Branch: refs/heads/master
Commit: d48f2667b822564260b6e5c9705a41e5dcd1c4fc
Parents: 8b8b668
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Aug 26 21:18:23 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Aug 26 21:21:15 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 pom.xml                                         |   9 +
 .../org/apache/tajo/catalog/CatalogServer.java  |   4 +-
 .../java/org/apache/tajo/ExecutionBlockId.java  |  97 +++
 .../src/main/java/org/apache/tajo/QueryId.java  | 180 +----
 .../java/org/apache/tajo/QueryIdFactory.java    |  96 ++-
 .../org/apache/tajo/QueryUnitAttemptId.java     | 144 +---
 .../main/java/org/apache/tajo/QueryUnitId.java  | 162 +---
 .../main/java/org/apache/tajo/SubQueryId.java   | 166 ----
 .../java/org/apache/tajo/conf/TajoConf.java     |   3 +-
 .../main/java/org/apache/tajo/util/TUtil.java   |   6 +-
 .../java/org/apache/tajo/util/TajoIdUtils.java  | 100 +--
 tajo-common/src/main/proto/TajoIdProtos.proto   |  11 +-
 .../org/apache/tajo/datum/TestBitDatum.java     |   2 +-
 tajo-core/tajo-core-backend/pom.xml             |   5 +-
 .../src/main/java/log4j.properties              |   3 +
 .../main/java/org/apache/tajo/cli/TajoCli.java  |  15 +-
 .../java/org/apache/tajo/client/TajoClient.java |  24 +-
 .../tajo/engine/query/QueryUnitRequestImpl.java |   4 +-
 .../apache/tajo/engine/query/ResultSetImpl.java |  41 +-
 .../ipc/protocolrecords/QueryUnitRequest.java   |   6 +-
 .../org/apache/tajo/master/ContainerProxy.java  | 409 +---------
 .../org/apache/tajo/master/ExecutionBlock.java  |  12 +-
 .../org/apache/tajo/master/GlobalEngine.java    | 244 ++----
 .../org/apache/tajo/master/GlobalPlanner.java   |  38 +-
 .../apache/tajo/master/TajoAsyncDispatcher.java | 234 ++++++
 .../apache/tajo/master/TajoContainerProxy.java  | 163 ++++
 .../java/org/apache/tajo/master/TajoMaster.java | 118 +--
 .../tajo/master/TajoMasterClientService.java    | 101 ++-
 .../apache/tajo/master/TajoMasterService.java   | 170 ++++
 .../tajo/master/TaskRunnerGroupEvent.java       |  12 +-
 .../tajo/master/TaskRunnerLauncherImpl.java     | 171 ----
 .../apache/tajo/master/TaskSchedulerImpl.java   |  71 +-
 .../apache/tajo/master/YarnContainerProxy.java  | 446 ++++++++++
 .../tajo/master/YarnTaskRunnerLauncherImpl.java | 208 +++++
 .../master/event/ContainerAllocationEvent.java  |  20 +-
 .../event/GrouppedContainerAllocatorEvent.java  |   6 +-
 .../tajo/master/event/QueryStartEvent.java      |  50 ++
 .../tajo/master/event/QuerySubQueryEvent.java   |  12 +-
 .../master/event/SubQueryCompletedEvent.java    |  14 +-
 .../event/SubQueryContainerAllocationEvent.java |   4 +-
 .../apache/tajo/master/event/SubQueryEvent.java |   8 +-
 .../tajo/master/event/SubQuerySucceeEvent.java  |   4 +-
 .../tajo/master/event/SubQueryTaskEvent.java    |   2 +-
 .../event/TaskAttemptStatusUpdateEvent.java     |   2 +-
 .../tajo/master/event/TaskCompletionEvent.java  |   2 +-
 .../tajo/master/event/TaskFatalErrorEvent.java  |   2 +-
 .../tajo/master/event/TaskRequestEvent.java     |  11 +-
 .../tajo/master/event/TaskScheduleEvent.java    |   2 +-
 .../tajo/master/event/TaskSchedulerEvent.java   |  12 +-
 .../apache/tajo/master/querymaster/Query.java   |  54 +-
 .../master/querymaster/QueryInProgress.java     | 285 +++++++
 .../tajo/master/querymaster/QueryInfo.java      | 127 +++
 .../tajo/master/querymaster/QueryJobEvent.java  |  43 +
 .../master/querymaster/QueryJobManager.java     | 172 ++++
 .../tajo/master/querymaster/QueryMaster.java    | 809 ++++---------------
 .../querymaster/QueryMasterClientService.java   | 197 -----
 .../master/querymaster/QueryMasterManager.java  | 353 --------
 .../querymaster/QueryMasterManagerService.java  | 116 ---
 .../master/querymaster/QueryMasterRunner.java   |  13 +-
 .../master/querymaster/QueryMasterTask.java     | 445 ++++++++++
 .../tajo/master/querymaster/QueryUnit.java      |  12 +-
 .../master/querymaster/QueryUnitAttempt.java    |  13 +-
 .../tajo/master/querymaster/Repartitioner.java  |  22 +-
 .../tajo/master/querymaster/SubQuery.java       |  48 +-
 .../tajo/master/rm/RMContainerAllocator.java    | 208 -----
 .../tajo/master/rm/TajoWorkerContainer.java     | 120 +++
 .../tajo/master/rm/TajoWorkerContainerId.java   |  47 ++
 .../master/rm/TajoWorkerResourceManager.java    | 394 +++++++++
 .../apache/tajo/master/rm/WorkerResource.java   | 194 +++++
 .../tajo/master/rm/WorkerResourceManager.java   |  60 ++
 .../org/apache/tajo/master/rm/WorkerStatus.java |  25 +
 .../master/rm/YarnRMContainerAllocator.java     | 236 ++++++
 .../tajo/master/rm/YarnTajoResourceManager.java | 337 ++++++++
 .../apache/tajo/util/ApplicationIdUtils.java    |  41 +
 .../apache/tajo/webapp/StaticHttpServer.java    |  26 +-
 .../tajo/worker/AbstractResourceAllocator.java  |  79 ++
 .../apache/tajo/worker/ResourceAllocator.java   |  27 +
 .../tajo/worker/TajoResourceAllocator.java      | 346 ++++++++
 .../java/org/apache/tajo/worker/TajoWorker.java | 424 ++++++++++
 .../tajo/worker/TajoWorkerClientService.java    | 210 +++++
 .../tajo/worker/TajoWorkerManagerService.java   | 234 ++++++
 .../main/java/org/apache/tajo/worker/Task.java  |  43 +-
 .../java/org/apache/tajo/worker/TaskRunner.java | 256 +++---
 .../apache/tajo/worker/TaskRunnerManager.java   | 108 +++
 .../tajo/worker/YarnResourceAllocator.java      | 106 +++
 .../retriever/AdvancedDataRetriever.java        |  10 +-
 .../src/main/proto/ClientProtocol.proto         |  12 +-
 .../src/main/proto/ClientProtos.proto           |  10 +-
 .../main/proto/QueryMasterClientProtocol.proto  |   2 +-
 .../main/proto/QueryMasterManagerProtocol.proto |  50 --
 .../src/main/proto/QueryMasterProtocol.proto    | 132 ---
 .../src/main/proto/TajoIdProtos.proto           |  11 +-
 .../main/proto/TajoMasterClientProtocol.proto   |   5 +-
 .../src/main/proto/TajoMasterProtocol.proto     |  98 +++
 .../src/main/proto/TajoWorkerProtocol.proto     | 137 ++++
 .../src/main/resources/log4j.properties         |   3 +
 .../src/main/resources/tajo-default.xml         |  68 ++
 .../src/test/java/log4j.properties              |   3 +
 .../apache/tajo/LocalTajoTestingUtility.java    |  13 +-
 .../org/apache/tajo/MiniTajoYarnCluster.java    |   3 -
 .../org/apache/tajo/TajoTestingCluster.java     | 120 +--
 .../org/apache/tajo/TestQueryIdFactory.java     |   7 +-
 .../test/java/org/apache/tajo/TestTajoIds.java  |  78 +-
 .../test/java/org/apache/tajo/TpchTestBase.java |  17 +-
 .../plan/global/TestGlobalQueryPlanner.java     |   1 -
 .../global/TestGlobalQueryOptimizer.java        |  15 +-
 .../planner/physical/TestPhysicalPlanner.java   |   2 -
 .../apache/tajo/master/TestRepartitioner.java   |   7 +-
 .../org/apache/tajo/worker/TaskRunnerTest.java  |  18 +-
 .../tajo/worker/TestRangeRetrieverHandler.java  |   2 -
 .../worker/dataserver/TestHttpDataServer.java   |  12 +-
 .../src/test/resources/tajo-default.xml         |   6 +
 tajo-core/tajo-core-pullserver/pom.xml          |  11 +-
 .../tajo/pullserver/HttpDataServerHandler.java  |   9 +-
 .../tajo/pullserver/PullServerAuxService.java   |  15 +-
 .../tajo/pullserver/TajoPullServerService.java  | 652 +++++++++++++++
 .../retriever/AdvancedDataRetriever.java        |  23 +-
 .../java/org/apache/tajo/storage/CSVFile.java   | 187 ++---
 .../org/apache/tajo/storage/MergeScanner.java   |   8 +-
 tajo-dist/src/main/bin/start-tajo.sh            |   8 +
 tajo-dist/src/main/bin/stop-tajo.sh             |  11 +-
 tajo-dist/src/main/bin/tajo                     |   4 +
 tajo-dist/src/main/bin/tajo-config.sh           |  15 +-
 tajo-dist/src/main/bin/tajo-daemon.sh           |  12 +-
 tajo-dist/src/main/bin/tajo-daemons.sh          |  68 ++
 tajo-dist/src/main/conf/tajo-env.sh             |  13 +-
 tajo-dist/src/main/conf/workers                 |   1 +
 .../org/apache/tajo/rpc/NettyServerBase.java    |   1 -
 .../apache/tajo/rpc/ProtoAsyncRpcClient.java    |   5 +-
 .../apache/tajo/rpc/ProtoAsyncRpcServer.java    |   3 +-
 .../apache/tajo/rpc/ProtoBlockingRpcClient.java |  12 +-
 .../java/org/apache/tajo/util/NetUtils.java     |   2 +-
 133 files changed, 7812 insertions(+), 3948 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de7d746..c6b3f84 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,8 @@ Release 0.2.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)
+
     TAJO-84: Task scheduling with considering disk load balance. (jinho)
 
     TAJO-123: Clean up the logical plan's json format. (hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e1e7b7d..07b05a1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -91,6 +91,14 @@
       <plugins>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-surefire-report-plugin</artifactId>
+          <version>2.15</version>
+          <configuration>
+            <aggregate>true</aggregate>
+          </configuration>
+        </plugin>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
           <version>2.5.1</version>
           <configuration>
@@ -297,6 +305,7 @@
           <numUnapprovedLicenses>0</numUnapprovedLicenses>
           <excludes>
             <exclude>CHANGES.txt</exclude>
+            <exclude>**/workers</exclude>
             <exclude>**/*.tql</exclude>
             <exclude>**/*.sql</exclude>
             <exclude>**/*.schema</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
index dd36c3e..f0a0584 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
@@ -128,7 +128,9 @@ public class CatalogServer extends AbstractService {
     // Creation of a HSA will force a resolve.
     InetSocketAddress initIsa = NetUtils.createSocketAddr(serverAddr);
     try {
-      this.rpcServer = new ProtoBlockingRpcServer(CatalogProtocol.class, handler, initIsa);
+      this.rpcServer = new ProtoBlockingRpcServer(
+          CatalogProtocol.class,
+          handler, initIsa);
       this.rpcServer.start();
 
       this.bindAddress = NetUtils.getConnectAddress(this.rpcServer.getListenAddress());

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java b/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
new file mode 100644
index 0000000..2dc4441
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+public class ExecutionBlockId implements Comparable<ExecutionBlockId> {
+  public static final String EB_ID_PREFIX = "eb";
+  private QueryId queryId;
+  private int id;
+
+  public ExecutionBlockId(QueryId queryId, int id) {
+    this.queryId = queryId;
+    this.id = id;
+  }
+
+  public ExecutionBlockId(TajoIdProtos.ExecutionBlockIdProto proto) {
+    this(new QueryId(proto.getQueryId()),  proto.getId());
+  }
+
+//  public ExecutionBlockId(String idStr) {
+//    String[] tokens = idStr.split(QueryId.SEPARATOR);
+//    if(tokens.length < 3) {
+//      throw new RuntimeException("Wrong ExecutionBlockId format[" + idStr + "]");
+//    }
+//
+//    this.queryId = new QueryId(tokens);
+//    this.id = Integer.parseInt(tokens[3]);
+//  }
+
+  @Override
+  public String toString() {
+      return EB_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
+  }
+
+  @Override
+  public int compareTo(ExecutionBlockId executionBlockId) {
+    int result = queryId.compareTo(executionBlockId.queryId);
+    if (result == 0) {
+      return id - executionBlockId.id;
+    } else {
+      return result;
+    }
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (this == obj) {
+      return true;
+    }
+    if(!(obj instanceof ExecutionBlockId)) {
+      return false;
+    }
+    return compareTo((ExecutionBlockId)obj) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+  public TajoIdProtos.ExecutionBlockIdProto getProto() {
+    return TajoIdProtos.ExecutionBlockIdProto.newBuilder()
+        .setQueryId(queryId.getProto())
+        .setId(id)
+        .build();
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public int getId() {
+    return id;
+  }
+
+  public String toStringNoPrefix() {
+    return queryId.toStringNoPrefix() + QueryId.SEPARATOR + QueryIdFactory.EB_ID_FORMAT.format(id);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryId.java b/tajo-common/src/main/java/org/apache/tajo/QueryId.java
index 5dbbaca..af1ae56 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryId.java
@@ -18,172 +18,72 @@
 
 package org.apache.tajo;
 
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
-
-import java.text.NumberFormat;
-
-/**
- * QueryId represents a unique identifier of a query.
- */
 public class QueryId implements Comparable<QueryId> {
-  public static final String PREFIX = "q";
-  public static final String SEPARATOR = "_";
-
-  ApplicationAttemptIdProto proto = ApplicationAttemptIdProto
-      .getDefaultInstance();
-  ApplicationAttemptIdProto.Builder builder = null;
-  boolean viaProto = false;
+  public static String SEPARATOR = "_";
+  public static final String QUERY_ID_PREFIX = "q";
 
-  private ApplicationId applicationId = null;
+  private String id;
+  private int seq;
 
-  public QueryId() {
-    builder = ApplicationAttemptIdProto.newBuilder();
+  public QueryId(String id, int seq) {
+    this.id = id;
+    this.seq = seq;
   }
 
-  public QueryId(ApplicationAttemptIdProto proto) {
-    this.proto = proto;
-    viaProto = true;
+  public QueryId(TajoIdProtos.QueryIdProto queryId) {
+    this(queryId.getId(), queryId.getSeq());
   }
 
-  public synchronized ApplicationAttemptIdProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
+  public String getId() {
+    return id;
   }
 
-  private synchronized void mergeLocalToBuilder() {
-    if (this.applicationId != null
-        && !((ApplicationIdPBImpl) applicationId).getProto().equals(
-        builder.getApplicationId())) {
-      builder.setApplicationId(convertToProtoFormat(this.applicationId));
-    }
+  public int getSeq() {
+    return seq;
   }
 
-  private synchronized void mergeLocalToProto() {
-    if (viaProto)
-      maybeInitBuilder();
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
+  @Override
+  public String toString() {
+    return QUERY_ID_PREFIX + SEPARATOR + toStringNoPrefix();
   }
 
-  private synchronized void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = ApplicationAttemptIdProto.newBuilder(proto);
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
     }
-    viaProto = false;
-  }
-
-  public synchronized int getAttemptId() {
-    ApplicationAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
-    return (p.getAttemptId());
-  }
-
-  public synchronized void setAttemptId(int attemptId) {
-    maybeInitBuilder();
-    builder.setAttemptId((attemptId));
-  }
-
-  public synchronized ApplicationId getApplicationId() {
-    ApplicationAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.applicationId != null) {
-      return this.applicationId;
+    if (this == obj) {
+      return true;
     }
-    if (!p.hasApplicationId()) {
-      return null;
+    if(!(obj instanceof QueryId)) {
+      return false;
     }
-    this.applicationId = convertFromProtoFormat(p.getApplicationId());
-    return this.applicationId;
-  }
-
-  public synchronized void setApplicationId(ApplicationId appId) {
-    maybeInitBuilder();
-    if (appId == null)
-      builder.clearApplicationId();
-    this.applicationId = appId;
-  }
-
-  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
-    return new ApplicationIdPBImpl(p);
+    return compareTo((QueryId)obj) == 0;
   }
 
-  public static ApplicationIdProto convertToProtoFormat(ApplicationId t) {
-    return ((ApplicationIdPBImpl)t).getProto();
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
   }
 
   @Override
   public int compareTo(QueryId queryId) {
-    int compVal = getApplicationId().compareTo(queryId.getApplicationId());
-    if (compVal != 0) {
-      return compVal;
+    int result = id.compareTo(queryId.id);
+    if (result == 0) {
+      return seq - queryId.seq;
     } else {
-      return getAttemptId() - queryId.getAttemptId();
+      return result;
     }
   }
 
-  static final ThreadLocal<NumberFormat> appIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(4);
-          return fmt;
-        }
-      };
-
-  static final ThreadLocal<NumberFormat> attemptIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
-
-  @Override
-  public int hashCode() {
-    // Generated by eclipse.
-    final int prime = 31;
-    int result = 1;
-    ApplicationId appId = getApplicationId();
-    result = prime * result +  appId.hashCode();
-    result = prime * result + getAttemptId();
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    QueryId other = (QueryId) obj;
-    if (!this.getApplicationId().equals(other.getApplicationId()))
-      return false;
-    if (this.getAttemptId() != other.getAttemptId())
-      return false;
-    return true;
+  public TajoIdProtos.QueryIdProto getProto() {
+    return TajoIdProtos.QueryIdProto.newBuilder()
+        .setId(id)
+        .setSeq(seq)
+        .build();
   }
 
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(PREFIX).append(SEPARATOR)
-    .append(this.getApplicationId().getClusterTimestamp()).append(SEPARATOR)
-    .append(appIdFormat.get().format(this.getApplicationId().getId()))
-    .append(SEPARATOR)
-        .append(attemptIdFormat.get().format(getAttemptId()));
-    return sb.toString();
+  public String toStringNoPrefix() {
+    return id + SEPARATOR + QueryIdFactory.ID_FORMAT.format(seq);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java b/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
index b1a6ab4..90533e3 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
@@ -18,39 +18,97 @@
 
 package org.apache.tajo;
 
-import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.tajo.util.TajoIdUtils;
 
+import java.text.DecimalFormat;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 public class QueryIdFactory {
-  private static AtomicInteger nextId = 
-      new AtomicInteger(-1);
-  
-  public static void reset() {
-    nextId.set(-1);
+  public static final QueryId NULL_QUERY_ID = newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0), 0);
+
+  public static DecimalFormat ID_FORMAT = new DecimalFormat("0000");
+
+  public static DecimalFormat EB_ID_FORMAT = new DecimalFormat("000000");
+
+  public static DecimalFormat QU_ID_FORMAT = new DecimalFormat("000000");
+
+  public static DecimalFormat ATTEMPT_ID_FORMAT = new DecimalFormat("00");
+
+  private static Map<String, AtomicInteger> queryNexIdMap = new HashMap<String, AtomicInteger>();
+
+  private static AtomicInteger nextId = new AtomicInteger(0);
+
+  /*
+   * <ul>
+   * <li>QueryId == q_{masterId}_{seq}</li>
+   * <li>masterId == tajoMasterId or YarnResourceManagerId</li>
+   * <li>seq = TajoSeq or YarnSeq</li>
+   * </ul>
+   */
+  public synchronized static QueryId newQueryId(String seedQueryId) {
+    AtomicInteger queryNextId = queryNexIdMap.get(seedQueryId);
+    if(queryNextId == null) {
+      queryNextId = new AtomicInteger(0);
+      queryNexIdMap.put(seedQueryId, queryNextId);
+    }
+    if(isYarnId(seedQueryId)) {
+      String[] tokens = seedQueryId.split("_");
+      return new QueryId(tokens[1], Integer.parseInt(tokens[2]));
+    } else {
+      int seq = queryNextId.incrementAndGet();
+      if(seq >= 10000) {
+        queryNextId.set(0);
+        seq = queryNextId.incrementAndGet();
+      }
+
+      return new QueryId(seedQueryId, seq);
+    }
+  }
+
+  public synchronized static QueryId newQueryId(long timestamp, int seq) {
+    return new QueryId(String.valueOf(timestamp), seq);
   }
 
+  /**
+   * for test
+   * @return
+   */
   public synchronized static QueryId newQueryId() {
-    int idInt = nextId.incrementAndGet();
-    return TajoIdUtils.createQueryId(BuilderUtils.newApplicationId(
-        System.currentTimeMillis(), idInt), idInt);
+    return newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0));
   }
-  
-  public synchronized static SubQueryId newSubQueryId(QueryId queryId) {
-    return TajoIdUtils.createSubQueryId(queryId, nextId.incrementAndGet());
+
+  public synchronized static QueryId newQueryId(String seedQueryId, int seq) {
+    if(isYarnId(seedQueryId)) {
+      String[] tokens = seedQueryId.split("_");
+      return new QueryId(tokens[1], Integer.parseInt(tokens[2]));
+    } else {
+      return new QueryId(seedQueryId, seq);
+    }
   }
-  
-  public synchronized static QueryUnitId newQueryUnitId(SubQueryId subQueryId) {
-    return new QueryUnitId(subQueryId, nextId.incrementAndGet());
+
+  private static boolean isYarnId(String id) {
+    return id.startsWith("application");
+  }
+
+  public synchronized static ExecutionBlockId newExecutionBlockId(QueryId queryId) {
+    return new ExecutionBlockId(queryId, nextId.incrementAndGet());
+  }
+
+  public synchronized static ExecutionBlockId newExecutionBlockId(QueryId queryId, int id) {
+    return new ExecutionBlockId(queryId, id);
+  }
+
+  public synchronized static QueryUnitId newQueryUnitId(ExecutionBlockId executionBlockId) {
+    return new QueryUnitId(executionBlockId, nextId.incrementAndGet());
   }
 
-  public synchronized static QueryUnitId newQueryUnitId(SubQueryId subQueryId, int taskId) {
-    return new QueryUnitId(subQueryId, taskId);
+  public synchronized static QueryUnitId newQueryUnitId(ExecutionBlockId executionBlockId, int id) {
+    return new QueryUnitId(executionBlockId, id);
   }
 
-  public synchronized static QueryUnitAttemptId newQueryUnitAttemptId(
-      final QueryUnitId queryUnitId, final int attemptId) {
+  public synchronized static QueryUnitAttemptId newQueryUnitAttemptId(QueryUnitId queryUnitId, final int attemptId) {
     return new QueryUnitAttemptId(queryUnitId, attemptId);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java b/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
index 3ef436e..98ba5d1 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
@@ -18,139 +18,75 @@
 
 package org.apache.tajo;
 
-import com.google.common.base.Objects;
-import org.apache.tajo.TajoIdProtos.QueryUnitAttemptIdProto;
-import org.apache.tajo.TajoIdProtos.QueryUnitAttemptIdProtoOrBuilder;
-import org.apache.tajo.common.ProtoObject;
+public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId> {
+  public static final String QUA_ID_PREFIX = "ta";
 
-import java.text.NumberFormat;
+  private QueryUnitId queryUnitId;
+  private int id;
 
-public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId>, ProtoObject<QueryUnitAttemptIdProto> {
-  private static final String PREFIX="ta";
-
-  private static final NumberFormat format = NumberFormat.getInstance();
-  static {
-    format.setGroupingUsed(false);
-    format.setMinimumIntegerDigits(2);
+  public QueryUnitId getQueryUnitId() {
+    return queryUnitId;
   }
 
-  private QueryUnitId queryUnitId = null;
-  private int id = -1;
-  private String finalId = null;
-
-  private QueryUnitAttemptIdProto proto =
-      QueryUnitAttemptIdProto.getDefaultInstance();
-  private QueryUnitAttemptIdProto.Builder builder = null;
-  private boolean viaProto = false;
-
-  public QueryUnitAttemptId() {
-    builder = QueryUnitAttemptIdProto.newBuilder();
+  public int getId() {
+    return id;
   }
 
-  public QueryUnitAttemptId(final QueryUnitId queryUnitId, final int id) {
-    this.queryUnitId = queryUnitId;
+  public void setId(int id) {
     this.id = id;
   }
 
-  public QueryUnitAttemptId(QueryUnitAttemptIdProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  public QueryUnitAttemptId(final String finalId) {
-    this.finalId = finalId;
-    int i = finalId.lastIndexOf(QueryId.SEPARATOR);
-    this.queryUnitId = new QueryUnitId(finalId.substring(0, i));
-    this.id = Integer.valueOf(finalId.substring(i+1));
-  }
-
-  public int getId() {
-    QueryUnitAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.id != -1) {
-      return this.id;
-    }
-    if (!p.hasId()) {
-      return -1;
-    }
-    this.id = p.getId();
-    return id;
-  }
-
-  public QueryUnitId getQueryUnitId() {
-    QueryUnitAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.queryUnitId != null) {
-      return this.queryUnitId;
-    }
-    if (!p.hasId()) {
-      return null;
-    }
-    this.queryUnitId = new QueryUnitId(p.getQueryUnitId());
-    return queryUnitId;
+  public QueryUnitAttemptId(QueryUnitId queryUnitId, int id) {
+    this.queryUnitId = queryUnitId;
+    this.id = id;
   }
 
-  public QueryId getQueryId() {
-    return this.getQueryUnitId().getQueryId();
+  public QueryUnitAttemptId(TajoIdProtos.QueryUnitAttemptIdProto proto) {
+    this(new QueryUnitId(proto.getQueryUnitId()), proto.getId());
   }
 
-  public SubQueryId getSubQueryId() {
-    return this.getQueryUnitId().getSubQueryId();
+  public TajoIdProtos.QueryUnitAttemptIdProto getProto() {
+    return TajoIdProtos.QueryUnitAttemptIdProto.newBuilder()
+        .setQueryUnitId(queryUnitId.getProto())
+        .setId(id)
+        .build();
   }
 
   @Override
-  public final String toString() {
-    if (finalId == null) {
-      StringBuilder sb = new StringBuilder(PREFIX);
-      SubQueryId subQueryId = getQueryUnitId().getSubQueryId();
-      QueryId appId = subQueryId.getQueryId();
-      sb.append(QueryId.SEPARATOR).append(appId.getApplicationId().getClusterTimestamp())
-          .append(QueryId.SEPARATOR).append(QueryId.appIdFormat.get().format(appId.getApplicationId().getId()))
-          .append(QueryId.SEPARATOR).append(QueryId.attemptIdFormat.get().format(appId.getAttemptId()))
-          .append(QueryId.SEPARATOR).append(SubQueryId.subQueryIdFormat.get().format(subQueryId.getId()))
-          .append(QueryId.SEPARATOR).append(QueryUnitId.queryUnitIdFormat.get().format(getQueryUnitId().getId()))
-          .append(QueryId.SEPARATOR).append(format.format(getId()));
-      finalId = sb.toString();
+  public int compareTo(QueryUnitAttemptId queryUnitAttemptId) {
+    int result = queryUnitId.compareTo(queryUnitAttemptId.queryUnitId);
+    if (result == 0) {
+      return id - queryUnitAttemptId.id;
+    } else {
+      return result;
     }
-    return this.finalId;
   }
 
   @Override
-  public final boolean equals(final Object o) {
-    if (o instanceof QueryUnitAttemptId) {
-      QueryUnitAttemptId other = (QueryUnitAttemptId) o;
-      return this.toString().equals(other.toString());
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (this == obj) {
+      return true;
     }
-    return false;
+    if(!(obj instanceof QueryUnitAttemptId)) {
+      return false;
+    }
+    return compareTo((QueryUnitAttemptId)obj) == 0;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hashCode(getQueryUnitId(), getId());
+    return toString().hashCode();
   }
 
   @Override
-  public int compareTo(QueryUnitAttemptId o) {
-    return this.getId() - o.getId();
-  }
-
-  private void mergeLocalToBuilder() {
-    if (builder == null) {
-      builder = QueryUnitAttemptIdProto.newBuilder(proto);
-    }
-    if (this.queryUnitId != null) {
-      builder.setQueryUnitId(queryUnitId.getProto());
-    }
-    if (this.id != -1) {
-      builder.setId(id);
-    }
+  public String toString() {
+    return QUA_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
   }
 
-  @Override
-  public QueryUnitAttemptIdProto getProto() {
-    if (!viaProto) {
-      mergeLocalToBuilder();
-      proto = builder.build();
-      viaProto = true;
-    }
-    return proto;
+  public String toStringNoPrefix() {
+    return queryUnitId.toStringNoPrefix() + QueryId.SEPARATOR + QueryIdFactory.ATTEMPT_ID_FORMAT.format(id);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java b/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
index 4826691..21addf9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
@@ -18,145 +18,71 @@
 
 package org.apache.tajo;
 
-import com.google.common.base.Objects;
-import org.apache.tajo.TajoIdProtos.QueryUnitIdProto;
-import org.apache.tajo.TajoIdProtos.QueryUnitIdProtoOrBuilder;
-import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.util.TajoIdUtils;
+public class QueryUnitId implements Comparable<QueryUnitId> {
+  public static final String QU_ID_PREFIX = "t";
 
-import java.text.NumberFormat;
+  private ExecutionBlockId executionBlockId;
+  private int id;
 
-public class QueryUnitId implements Comparable<QueryUnitId>,
-  ProtoObject<QueryUnitIdProto> {
-  private static final String PREFIX = "t";
-
-  static final ThreadLocal<NumberFormat> queryUnitIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
-  
-  private SubQueryId subQueryId = null;
-  private int id = -1;
-  private String finalId = null;
-
-  private QueryUnitIdProto proto = QueryUnitIdProto.getDefaultInstance();
-  private QueryUnitIdProto.Builder builder = null;
-  private boolean viaProto = false;
-  
-  public QueryUnitId() {
-    builder = QueryUnitIdProto.newBuilder();
-  }
-  
-  public QueryUnitId(final SubQueryId subQueryId,
-      final int id) {
-    this.subQueryId = subQueryId;
+  public QueryUnitId(ExecutionBlockId executionBlockId, int id) {
+    this.executionBlockId = executionBlockId;
     this.id = id;
   }
-  
-  public QueryUnitId(QueryUnitIdProto proto) {
-    this.proto = proto;
-    viaProto = true;
+
+  public QueryUnitId(TajoIdProtos.QueryUnitIdProto proto) {
+    this(new ExecutionBlockId(proto.getExecutionBlockId()), proto.getId());
   }
-  
-  public QueryUnitId(final String finalId) {
-    this.finalId = finalId;
-    int i = finalId.lastIndexOf(QueryId.SEPARATOR);
-    this.subQueryId = TajoIdUtils.newSubQueryId(finalId.substring(0, i));
-    this.id = Integer.valueOf(finalId.substring(i+1));
+
+  public ExecutionBlockId getExecutionBlockId() {
+    return executionBlockId;
   }
-  
+
   public int getId() {
-    QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.id != -1) {
-      return this.id;
-    }
-    if (!p.hasId()) {
-      return -1;
-    }
-    this.id = p.getId();
     return id;
   }
-  
-  public SubQueryId getSubQueryId() {
-    QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.subQueryId != null) {
-      return this.subQueryId;
-    }
-    if (!p.hasSubQueryId()) {
-      return null;
-    }
-    this.subQueryId = TajoIdUtils.newSubQueryId(p.getSubQueryId());
-    return this.subQueryId;
-  }
-  
-  public QueryId getQueryId() {
-    return this.getSubQueryId().getQueryId();
+
+  public TajoIdProtos.QueryUnitIdProto getProto() {
+    return TajoIdProtos.QueryUnitIdProto.newBuilder()
+        .setExecutionBlockId(executionBlockId.getProto())
+        .setId(id)
+        .build();
   }
-  
+
   @Override
-  public final String toString() {
-    if (finalId == null) {
-      StringBuilder sb = new StringBuilder(PREFIX);
-      QueryId appId = getSubQueryId().getQueryId();
-      sb.append(QueryId.SEPARATOR).append(
-          appId.getApplicationId().getClusterTimestamp())
-      .append(QueryId.SEPARATOR).append(
-          QueryId.appIdFormat.get().format(appId.getApplicationId().getId()))
-      .append(QueryId.SEPARATOR).append(
-          QueryId.attemptIdFormat.get().format(appId.getAttemptId()))
-      .append(QueryId.SEPARATOR).append(
-          SubQueryId.subQueryIdFormat.get().format(getSubQueryId().getId()))
-      .append(QueryId.SEPARATOR).append(queryUnitIdFormat.get().format(getId()));
-      finalId = sb.toString();
+  public int compareTo(QueryUnitId queryUnitId) {
+    int result = executionBlockId.compareTo(queryUnitId.executionBlockId);
+    if (result == 0) {
+      return id - queryUnitId.id;
+    } else {
+      return result;
     }
-    return this.finalId;
   }
-  
+
   @Override
-  public final boolean equals(final Object o) {
-    if (o instanceof QueryUnitId) {
-      QueryUnitId other = (QueryUnitId) o;
-      return getSubQueryId().equals(other.getSubQueryId()) &&
-          getId() == other.getId();
-    }    
-    return false;
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+    if (this == obj) {
+      return true;
+    }
+    if(!(obj instanceof QueryUnitId)) {
+      return false;
+    }
+    return compareTo((QueryUnitId)obj) == 0;
   }
-  
+
   @Override
   public int hashCode() {
-    return Objects.hashCode(getSubQueryId(), getId());
+    return toString().hashCode();
   }
 
   @Override
-  public final int compareTo(final QueryUnitId o) {
-    return this.toString().compareTo(o.toString());
-  }
-  
-  private void mergeLocalToBuilder() {
-    if (builder == null) {
-      builder = QueryUnitIdProto.newBuilder(proto);
-    }
-    if (this.subQueryId != null) {
-      builder.setSubQueryId(subQueryId.getProto());
-    }
-    if (this.id != -1) {
-      builder.setId(id);
-    }
+  public String toString() {
+    return QU_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
   }
 
-  @Override
-  public QueryUnitIdProto getProto() {
-    if (!viaProto) {
-      mergeLocalToBuilder();
-      proto = builder.build();
-      viaProto = true;
-    }
-    return proto;
+  public String toStringNoPrefix() {
+    return executionBlockId.toStringNoPrefix() + QueryId.SEPARATOR + QueryIdFactory.QU_ID_FORMAT.format(id);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java b/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java
deleted file mode 100644
index 2a11f38..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo;
-
-import org.apache.tajo.TajoIdProtos.SubQueryIdProto;
-import org.apache.tajo.TajoIdProtos.SubQueryIdProtoOrBuilder;
-
-import java.text.NumberFormat;
-
-public class SubQueryId implements Comparable<SubQueryId> {
-  public static final String PREFIX = "sq";
-
-  static final ThreadLocal<NumberFormat> subQueryIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(2);
-          return fmt;
-        }
-      };
-
-  private SubQueryIdProto proto = SubQueryIdProto.getDefaultInstance();
-  private SubQueryIdProto.Builder builder = null;
-  private boolean viaProto = false;
-
-  private QueryId queryId = null;
-
-  public SubQueryId() {
-    builder = SubQueryIdProto.newBuilder(proto);
-  }
-
-  public SubQueryId(SubQueryIdProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  /**
-   * @return the subquery number.
-   */
-  public synchronized int getId() {
-    SubQueryIdProtoOrBuilder p = viaProto ? proto : builder;
-    return (p.getId());
-  }
-
-  public synchronized void setId(int id) {
-    maybeInitBuilder();
-    builder.setId((id));
-  }
-
-  /**
-   * @return the associated <code>QueryId</code>
-   */
-  public synchronized QueryId getQueryId() {
-    SubQueryIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.queryId != null) {
-      return this.queryId;
-    }
-    if (!p.hasQueryId()) {
-      return null;
-    }
-    queryId = new QueryId(p.getQueryId());
-    return queryId;
-  }
-
-  public synchronized void setQueryId(QueryId queryId) {
-    maybeInitBuilder();
-    if (queryId == null)
-      builder.clearQueryId();
-    this.queryId = queryId;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + getId();
-    result = prime * result + getQueryId().hashCode();
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    SubQueryId other = (SubQueryId) obj;
-    if (getId() != other.getId())
-      return false;
-    if (!getQueryId().equals(other.getQueryId()))
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder(PREFIX);
-    QueryId queryId = getQueryId();
-    builder.append(QueryId.SEPARATOR).append(queryId.getApplicationId().getClusterTimestamp());
-    builder.append(QueryId.SEPARATOR).append(
-        QueryId.appIdFormat.get().format(queryId.getApplicationId().getId()));
-    builder.append(QueryId.SEPARATOR).append(QueryId.attemptIdFormat.get().format(queryId.getAttemptId()))
-        .append(QueryId.SEPARATOR)
-    .append(subQueryIdFormat.get().format(getId()));
-    return builder.toString();
-  }
-
-  @Override
-  public int compareTo(SubQueryId other) {
-    int queryIdComp = this.getQueryId().compareTo(other.getQueryId());
-    if (queryIdComp == 0) {
-      return this.getId() - other.getId();
-    } else {
-      return queryIdComp;
-    }
-  }
-
-  public synchronized SubQueryIdProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  private synchronized void mergeLocalToBuilder() {
-    if (this.queryId != null
-        && !this.queryId.getProto().equals(builder.getQueryId())) {
-      builder.setQueryId(queryId.getProto());
-    }
-  }
-
-  private synchronized void mergeLocalToProto() {
-    if (viaProto)
-      maybeInitBuilder();
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private synchronized void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = SubQueryIdProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 21d0c63..2fe64e6 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -66,8 +66,7 @@ public class TajoConf extends YarnConfiguration {
     // Service Addresses
     TASKRUNNER_LISTENER_ADDRESS("tajo.master.taskrunnerlistener.addr", "0.0.0.0:0"), // used internally
     CLIENT_SERVICE_ADDRESS("tajo.master.clientservice.addr", "127.0.0.1:9004"),
-    QUERY_MASTER_MANAGER_SERVICE_ADDRESS("tajo.master.querymastermanager.addr", "127.0.0.1:9005"),
-    QUERY_MASTER_CLIENT_SERVICE_ADDRESS("tajo.qmm.client.addr", "0.0.0.0:0"),
+    TAJO_MASTER_SERVICE_ADDRESS("tajo.master.manager.addr", "127.0.0.1:9005"),
 
     //////////////////////////////////
     // Catalog Configuration

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
index 7eaade8..05c0972 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
@@ -108,11 +108,11 @@ public class TUtil {
     return list;
   }
 
-  public  static QueryUnitAttemptId newQueryUnitAttemptId() {
+  public static QueryUnitAttemptId newQueryUnitAttemptId() {
     return QueryIdFactory.newQueryUnitAttemptId(
         QueryIdFactory.newQueryUnitId(
-                QueryIdFactory.newSubQueryId(
-                    QueryIdFactory.newQueryId())), 0);
+            QueryIdFactory.newExecutionBlockId(
+                QueryIdFactory.newQueryId())), 0);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
index 9dfbfbc..3b4bd51 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
@@ -18,104 +18,22 @@
 
 package org.apache.tajo.util;
 
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.TajoIdProtos.SubQueryIdProto;
 
-import java.util.Iterator;
-
-import static org.apache.hadoop.yarn.util.StringHelper._split;
+import java.text.DecimalFormat;
 
 public class TajoIdUtils {
-  public static final String YARN_APPLICATION_PREFIX = "application";
-  public static final String YARN_CONTAINER_PREFIX = "container";
-  public static final String YARN_APPLICATION_ATTEMPT_PREFIX = "appattempt";
-
-  /** It is mainly for DDL statements which don's have any query id. */
-  public static final QueryId NullQueryId =
-      TajoIdUtils.createQueryId(BuilderUtils.newApplicationId(0, 0), 0);
+  public static DecimalFormat MASTER_ID_FORMAT = new DecimalFormat("0000000000000");
 
-  public static QueryId createQueryId(ApplicationId appId, int attemptId) {
-    return newQueryId(appId, attemptId);
-  }
-
-  public static QueryId createQueryId(ApplicationAttemptId appAttemptId) {
-    QueryId queryId = new QueryId();
-    queryId.setApplicationId(appAttemptId.getApplicationId());
-    queryId.setAttemptId(appAttemptId.getAttemptId());
-    return queryId;
-  }
-
-  public static QueryId createQueryId(String queryId) {
-    String[] split = queryId.split(QueryId.SEPARATOR);
-    ApplicationId appId = BuilderUtils.newApplicationId(Long.valueOf(split[1]),
-        Integer.parseInt(split[2]));
-    int idInt = Integer.parseInt(split[3]);
-    return newQueryId(appId, idInt);
-  }
-
-  public static SubQueryId createSubQueryId(QueryId queryId,
-                                            int subQueryIdInt) {
-    return newSubQueryId(queryId, subQueryIdInt);
-  }
-
-  public static QueryId newQueryId(ApplicationId appId, int id) {
-    QueryId queryId = new QueryId();
-    queryId.setApplicationId(appId);
-    queryId.setAttemptId(id);
-    return queryId;
-  }
-
-  public static SubQueryId newSubQueryId(QueryId jobId, int id) {
-    SubQueryId taskId = new SubQueryId();
-    taskId.setQueryId(jobId);
-    taskId.setId(id);
-    return taskId;
-  }
-
-  public static SubQueryId newSubQueryId(String subQueryId) {
-    String [] split = subQueryId.split(QueryId.SEPARATOR);
-    ApplicationId appId = BuilderUtils.newApplicationId(Long.valueOf(split[1]),
-        Integer.valueOf(split[2]));
-    QueryId queryId = TajoIdUtils.createQueryId(appId, Integer.valueOf(split[3]));
-    return createSubQueryId(queryId, Integer.parseInt(split[4]));
-  }
-
-  public static SubQueryId newSubQueryId(SubQueryIdProto proto) {
-    SubQueryId subId = new SubQueryId(proto);
-    return subId;
-  }
+  public static ExecutionBlockId createExecutionBlockId(String idStr) {
+    String[] tokens = idStr.split("_");
 
-  public static ApplicationAttemptId toApplicationAttemptId(
-          String applicationAttmeptIdStr) {
-    //This methood from YARN.ConvertUtils
-    Iterator<String> it = _split(applicationAttmeptIdStr).iterator();
-    if (!it.next().equals(YARN_APPLICATION_ATTEMPT_PREFIX)) {
-      throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
-              + applicationAttmeptIdStr);
-    }
-    try {
-      return toApplicationAttemptId(it);
-    } catch (NumberFormatException n) {
-      throw new IllegalArgumentException("Invalid AppAttemptId: "
-              + applicationAttmeptIdStr, n);
-    }
+    return new ExecutionBlockId(new QueryId(tokens[1], Integer.parseInt(tokens[2])), Integer.parseInt(tokens[3]));
   }
 
-  private static ApplicationAttemptId toApplicationAttemptId(
-          Iterator<String> it) throws NumberFormatException {
-    //This methood from YARN.ConvertUtils
-    ApplicationId appId = Records.newRecord(ApplicationId.class);
-    appId.setClusterTimestamp(Long.parseLong(it.next()));
-    appId.setId(Integer.parseInt(it.next()));
-    ApplicationAttemptId appAttemptId = Records
-            .newRecord(ApplicationAttemptId.class);
-    appAttemptId.setApplicationId(appId);
-    appAttemptId.setAttemptId(Integer.parseInt(it.next()));
-    return appAttemptId;
+  public static QueryId parseQueryId(String idStr) {
+    String[] tokens = idStr.split("_");
+    return new QueryId(tokens[1], Integer.parseInt(tokens[2]));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/proto/TajoIdProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/proto/TajoIdProtos.proto b/tajo-common/src/main/proto/TajoIdProtos.proto
index 04c67f2..a87c825 100644
--- a/tajo-common/src/main/proto/TajoIdProtos.proto
+++ b/tajo-common/src/main/proto/TajoIdProtos.proto
@@ -21,15 +21,18 @@ option java_outer_classname = "TajoIdProtos";
 option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
-import "yarn_protos.proto";
+message QueryIdProto {
+    required string id = 1;
+    required int32 seq = 2;
+}
 
-message SubQueryIdProto {
-    required ApplicationAttemptIdProto queryId = 1;
+message ExecutionBlockIdProto {
+    required QueryIdProto queryId = 1;
     required int32 id = 2;
 }
 
 message QueryUnitIdProto {
-    required SubQueryIdProto subQueryId = 1;
+    required ExecutionBlockIdProto executionBlockId = 1;
     required int32 id = 2;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
index cbcaaf9..ba938b2 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
@@ -18,8 +18,8 @@
 
 package org.apache.tajo.datum;
 
-import org.junit.Test;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/pom.xml b/tajo-core/tajo-core-backend/pom.xml
index ba5ad9f..6c3351f 100644
--- a/tajo-core/tajo-core-backend/pom.xml
+++ b/tajo-core/tajo-core-backend/pom.xml
@@ -127,8 +127,8 @@
                 <argument>src/main/proto/tajo_protos.proto</argument>
                 <argument>src/main/proto/ClientProtos.proto</argument>
                 <argument>src/main/proto/QueryMasterClientProtocol.proto</argument>
-                <argument>src/main/proto/QueryMasterManagerProtocol.proto</argument>
-                <argument>src/main/proto/QueryMasterProtocol.proto</argument>
+                <argument>src/main/proto/TajoMasterProtocol.proto</argument>
+                <argument>src/main/proto/TajoWorkerProtocol.proto</argument>
                 <argument>src/main/proto/TajoMasterClientProtocol.proto</argument>
               </arguments>
             </configuration>
@@ -329,7 +329,6 @@
       <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
     </dependency>
-
     <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/log4j.properties b/tajo-core/tajo-core-backend/src/main/java/log4j.properties
index c1ac487..749124c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/main/java/log4j.properties
@@ -23,3 +23,6 @@ log4j.threshhold=ALL
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
index 2dd25de..55a97ca 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -25,6 +25,7 @@ import jline.console.history.PersistentHistory;
 import org.apache.commons.cli.*;
 import org.apache.commons.lang.StringUtils;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
@@ -34,7 +35,6 @@ import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.TajoIdUtils;
 
 import java.io.File;
 import java.io.InputStream;
@@ -264,6 +264,7 @@ public class TajoCli {
   }
 
   public int executeStatements(String line) throws Exception {
+
     String stripped;
     for (String statement : line.split(";")) {
       stripped = StringUtils.chomp(statement);
@@ -282,12 +283,12 @@ public class TajoCli {
         invokeCommand(cmds);
 
       } else { // submit a query to TajoMaster
-        ClientProtos.SubmitQueryResponse response = client.executeQuery(stripped);
+        ClientProtos.GetQueryStatusResponse response = client.executeQuery(stripped);
         if (response.getResultCode() == ClientProtos.ResultCode.OK) {
           QueryId queryId = null;
           try {
             queryId = new QueryId(response.getQueryId());
-            if (queryId.equals(TajoIdUtils.NullQueryId)) {
+            if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
               sout.println("OK");
             } else {
               getQueryResult(queryId);
@@ -298,9 +299,9 @@ public class TajoCli {
             }
           }
         } else {
-        if (response.hasErrorMessage()) {
-          sout.println(response.getErrorMessage());
-        }
+          if (response.hasErrorMessage()) {
+            sout.println(response.getErrorMessage());
+          }
         }
       }
     }
@@ -313,7 +314,7 @@ public class TajoCli {
 
   private void getQueryResult(QueryId queryId) {
     // if query is empty string
-    if (queryId.equals(TajoIdUtils.NullQueryId)) {
+    if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
index cd8706e..7e7b787 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
@@ -22,7 +22,9 @@ import com.google.protobuf.ServiceException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
@@ -30,7 +32,6 @@ import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
 import org.apache.tajo.engine.query.ResultSetImpl;
-import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.ipc.QueryMasterClientProtocol;
 import org.apache.tajo.ipc.QueryMasterClientProtocol.QueryMasterClientProtocolService;
@@ -38,8 +39,6 @@ import org.apache.tajo.ipc.TajoMasterClientProtocol;
 import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService;
 import org.apache.tajo.rpc.ProtoBlockingRpcClient;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -128,7 +127,7 @@ public class TajoClient {
    * In order to get the result, you should use {@link #getQueryResult(org.apache.tajo.QueryId)}
    * or {@link #getQueryResultAndWait(org.apache.tajo.QueryId)}.
    */
-  public ClientProtos.SubmitQueryResponse executeQuery(String tql) throws ServiceException {
+  public GetQueryStatusResponse executeQuery(String tql) throws ServiceException {
     QueryRequest.Builder builder = QueryRequest.newBuilder();
     builder.setQuery(tql);
 
@@ -147,9 +146,9 @@ public class TajoClient {
       throws ServiceException, IOException {
     QueryRequest.Builder builder = QueryRequest.newBuilder();
     builder.setQuery(sql);
-    SubmitQueryResponse response = tajoMasterService.submitQuery(null, builder.build());
+    GetQueryStatusResponse response = tajoMasterService.submitQuery(null, builder.build());
     QueryId queryId = new QueryId(response.getQueryId());
-    if (queryId.equals(TajoIdUtils.NullQueryId)) {
+    if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return this.createNullResultSet(queryId);
     } else {
       return this.getQueryResultAndWait(queryId);
@@ -171,6 +170,9 @@ public class TajoClient {
       String queryMasterHost = res.getQueryMasterHost();
       if(queryMasterHost != null && !queryMasterHost.isEmpty()) {
         connectionToQueryMaster(queryId, queryMasterHost, res.getQueryMasterPort());
+
+        QueryMasterClientProtocolService.BlockingInterface queryMasterService = queryMasterConnectionMap.get(queryId);
+        res = queryMasterService.getQueryStatus(null, builder.build());
       }
     }
     return new QueryStatus(res);
@@ -204,7 +206,7 @@ public class TajoClient {
 
   public ResultSet getQueryResult(QueryId queryId)
       throws ServiceException, IOException {
-      if (queryId.equals(TajoIdUtils.NullQueryId)) {
+      if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
         return createNullResultSet(queryId);
       }
 
@@ -214,14 +216,14 @@ public class TajoClient {
 
   public ResultSet getQueryResultAndWait(QueryId queryId)
       throws ServiceException, IOException {
-    if (queryId.equals(TajoIdUtils.NullQueryId)) {
+    if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return createNullResultSet(queryId);
     }
     QueryStatus status = getQueryStatus(queryId);
 
     while(status != null && isQueryRunnning(status.getState())) {
       try {
-        Thread.sleep(1000);
+        Thread.sleep(500);
       } catch (InterruptedException e) {
         e.printStackTrace();
       }
@@ -237,7 +239,7 @@ public class TajoClient {
       }
 
     } else {
-      LOG.warn("Query " + status.getQueryId() + ") failed: " + status.getState());
+      LOG.warn("Query (" + status.getQueryId() + ") failed: " + status.getState());
 
       //TODO throw SQLException(?)
       return createNullResultSet(queryId);
@@ -249,7 +251,7 @@ public class TajoClient {
   }
 
   public TableDesc getResultDesc(QueryId queryId) throws ServiceException {
-    if (queryId.equals(TajoIdUtils.NullQueryId)) {
+    if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
index 7d430c5..cf10a4f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
@@ -19,7 +19,9 @@
 package org.apache.tajo.engine.query;
 
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Fetch;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProtoOrBuilder;
 import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
 import org.apache.tajo.storage.Fragment;
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
index 808b910..8cbe956 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
@@ -16,9 +16,6 @@
  * limitations under the License.
  */
 
-/**
- * 
- */
 package org.apache.tajo.engine.query;
 
 import com.google.common.collect.Lists;
@@ -297,7 +294,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Array getArray(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -308,7 +304,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Array getArray(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -319,7 +314,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public InputStream getAsciiStream(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -330,7 +324,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public InputStream getAsciiStream(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -363,7 +356,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public BigDecimal getBigDecimal(int arg0, int arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -374,7 +366,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public BigDecimal getBigDecimal(String arg0, int arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -385,7 +376,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public InputStream getBinaryStream(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -396,7 +386,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public InputStream getBinaryStream(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -499,7 +488,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Reader getCharacterStream(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -510,7 +498,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Reader getCharacterStream(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -561,7 +548,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Date getDate(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -572,7 +558,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Date getDate(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -583,7 +568,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Date getDate(int arg0, Calendar arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -594,7 +578,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Date getDate(String arg0, Calendar arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -741,7 +724,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Reader getNCharacterStream(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -752,7 +734,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Reader getNCharacterStream(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -783,7 +764,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public String getNString(int fieldId) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -794,7 +774,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public String getNString(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -832,7 +811,6 @@ public class ResultSetImpl implements ResultSet {
   @Override
   public Object getObject(int arg0, Map<String, Class<?>> arg1)
       throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -844,7 +822,6 @@ public class ResultSetImpl implements ResultSet {
   @Override
   public Object getObject(String arg0, Map<String, Class<?>> arg1)
       throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -855,7 +832,6 @@ public class ResultSetImpl implements ResultSet {
    */
   public <T> T getObject(String arg0, Class<T> arg1)
       throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -866,10 +842,9 @@ public class ResultSetImpl implements ResultSet {
    */
   public <T> T getObject(int arg0, Class<T> arg1)
       throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
-  
+
   /*
    * (non-Javadoc)
    * 
@@ -907,7 +882,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public RowId getRowId(int fieldId) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -918,7 +892,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public RowId getRowId(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1007,7 +980,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Time getTime(int fieldId) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1018,7 +990,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Time getTime(String name) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1029,7 +1000,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Time getTime(int fieldId, Calendar arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1040,7 +1010,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Time getTime(String name, Calendar arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1051,7 +1020,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Timestamp getTimestamp(int fieldId) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1062,7 +1030,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Timestamp getTimestamp(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1073,7 +1040,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Timestamp getTimestamp(int fieldId, Calendar arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1084,7 +1050,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public Timestamp getTimestamp(String arg0, Calendar arg1) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1105,7 +1070,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public URL getURL(int fieldId) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1116,7 +1080,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public URL getURL(String arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 
@@ -1147,7 +1110,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public SQLWarning getWarnings() throws SQLException {
-    // TODO
     throw new UnsupportedException();
   }
 
@@ -1298,7 +1260,6 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public boolean relative(int arg0) throws SQLException {
-    // TODO Auto-generated method stub
     throw new UnsupportedException();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
index bb4008f..c6fc632 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
@@ -23,13 +23,13 @@ package org.apache.tajo.ipc.protocolrecords;
 
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
 import org.apache.tajo.storage.Fragment;
 
 import java.net.URI;
 import java.util.List;
 
-public interface QueryUnitRequest extends ProtoObject<QueryMasterProtocol.QueryUnitRequestProto> {
+public interface QueryUnitRequest extends ProtoObject<TajoWorkerProtocol.QueryUnitRequestProto> {
 
 	public QueryUnitAttemptId getId();
 	public List<Fragment> getFragments();
@@ -39,7 +39,7 @@ public interface QueryUnitRequest extends ProtoObject<QueryMasterProtocol.QueryU
 	public boolean isInterQuery();
 	public void setInterQuery();
 	public void addFetch(String name, URI uri);
-	public List<QueryMasterProtocol.Fetch> getFetches();
+	public List<TajoWorkerProtocol.Fetch> getFetches();
   public boolean shouldDie();
   public void setShouldDie();
 }


[4/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
new file mode 100644
index 0000000..0c1d056
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -0,0 +1,445 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.Clock;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.worker.AbstractResourceAllocator;
+import org.apache.tajo.worker.TajoResourceAllocator;
+import org.apache.tajo.worker.YarnResourceAllocator;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QueryMasterTask extends CompositeService {
+  private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
+
+  // query submission directory is private!
+  final public static FsPermission USER_DIR_PERMISSION =
+      FsPermission.createImmutable((short) 0700); // rwx--------
+
+  private QueryId queryId;
+
+  private QueryContext queryContext;
+
+  private QueryMaster.QueryMasterContext queryMasterContext;
+
+  private Query query;
+
+  private MasterPlan masterPlan;
+
+  private String logicalPlanJson;
+
+  private TajoAsyncDispatcher dispatcher;
+
+  private final long querySubmitTime;
+
+  private boolean isCreateTableStmt;
+
+  private Path outputPath;
+
+  private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+
+  private QueryConf queryConf;
+
+  private AtomicLong lastClientHeartbeat = new AtomicLong(-1);
+
+  private AbstractResourceAllocator resourceAllocator;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
+                         QueryId queryId, String logicalPlanJson) {
+    super(QueryMasterTask.class.getName());
+    this.queryMasterContext = queryMasterContext;
+    this.queryId = queryId;
+    this.logicalPlanJson = logicalPlanJson;
+    this.querySubmitTime = System.currentTimeMillis();
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    queryConf = new QueryConf(conf);
+    queryConf.addResource(new Path(QueryConf.QUERY_MASTER_FILENAME));
+    try {
+      queryContext = new QueryContext();
+      String resourceManagerClassName = conf.get("tajo.resource.manager",
+          TajoWorkerResourceManager.class.getCanonicalName());
+
+      if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+        resourceAllocator = new TajoResourceAllocator(queryContext);
+      } else {
+        resourceAllocator = new YarnResourceAllocator(queryContext);
+      }
+      addService(resourceAllocator);
+
+      dispatcher = new TajoAsyncDispatcher(queryId.toString());
+      addService(dispatcher);
+
+      dispatcher.register(SubQueryEventType.class, new SubQueryEventDispatcher());
+      dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
+      dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+      dispatcher.register(QueryFinishEvent.EventType.class, new QueryFinishEventHandler());
+      dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+
+      initStagingDir();
+
+      super.init(queryConf);
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  public boolean isStopped() {
+    return stopped.get();
+  }
+
+  @Override
+  public void start() {
+    startQuery();
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if(stopped.get()) {
+      return;
+    }
+    stopped.set(true);
+
+    LOG.info("====> Stopping QueryMasterTask:" + queryId);
+
+    queryMasterContext.getWorkerContext().getTajoMasterRpcClient()
+        .stopQueryMaster(null, queryId.getProto(), NullCallback.get());
+
+    super.stop();
+
+    LOG.info("====> Stopped QueryMasterTask:" + queryId);
+  }
+
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    ExecutionBlockId id = event.getExecutionBlockId();
+    query.getSubQuery(id).handleTaskRequestEvent(event);
+  }
+
+  private class SubQueryEventDispatcher implements EventHandler<SubQueryEvent> {
+    public void handle(SubQueryEvent event) {
+      ExecutionBlockId id = event.getSubQueryId();
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("SubQueryEventDispatcher:" + id + "," + event.getType());
+      }
+      //Query query = queryMasterTasks.get(id.getQueryId()).getQuery();
+      query.getSubQuery(id).handle(event);
+    }
+  }
+
+  private class TaskEventDispatcher
+      implements EventHandler<TaskEvent> {
+    public void handle(TaskEvent event) {
+      QueryUnitId taskId = event.getTaskId();
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
+      }
+      //Query query = queryMasterTasks.get(taskId.getExecutionBlockId().getQueryId()).getQuery();
+      QueryUnit task = query.getSubQuery(taskId.getExecutionBlockId()).
+          getQueryUnit(taskId);
+      task.handle(event);
+    }
+  }
+
+  private class TaskAttemptEventDispatcher
+      implements EventHandler<TaskAttemptEvent> {
+    public void handle(TaskAttemptEvent event) {
+      QueryUnitAttemptId attemptId = event.getTaskAttemptId();
+      //Query query = queryMasterTasks.get(attemptId.getQueryUnitId().getExecutionBlockId().getQueryId()).getQuery();
+      SubQuery subQuery = query.getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId());
+      QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
+      QueryUnitAttempt attempt = task.getAttempt(attemptId);
+      attempt.handle(event);
+    }
+  }
+
+  private class TaskSchedulerDispatcher
+      implements EventHandler<TaskSchedulerEvent> {
+    public void handle(TaskSchedulerEvent event) {
+      //Query query = queryMasterTasks.get(event.getExecutionBlockId().getQueryId()).getQuery();
+      SubQuery subQuery = query.getSubQuery(event.getExecutionBlockId());
+      subQuery.getTaskScheduler().handle(event);
+    }
+  }
+
+  private class QueryFinishEventHandler implements EventHandler<QueryFinishEvent> {
+    @Override
+    public void handle(QueryFinishEvent event) {
+      QueryId queryId = event.getQueryId();
+      LOG.info("Query end notification started for QueryId : " + queryId);
+      //QueryMaster must be lived until client fetching all query result data.
+    }
+  }
+
+  public synchronized void startQuery() {
+    LOG.info("Plan JSON:" + logicalPlanJson);
+    if(query != null) {
+      LOG.warn("Query already started");
+      return;
+    }
+
+    try {
+      LogicalRootNode logicalNodeRoot = (LogicalRootNode) CoreGsonHelper.fromJson(logicalPlanJson, LogicalNode.class);
+      LogicalNode[] scanNodes = PlannerUtil.findAllNodes(logicalNodeRoot, NodeType.SCAN);
+      if(scanNodes != null) {
+        for(LogicalNode eachScanNode: scanNodes) {
+          ScanNode scanNode = (ScanNode)eachScanNode;
+          tableDescMap.put(scanNode.getFromTable().getTableName(), scanNode.getFromTable().getTableDesc());
+        }
+      }
+      MasterPlan globalPlan = queryMasterContext.getGlobalPlanner().build(queryId, logicalNodeRoot);
+      this.masterPlan = queryMasterContext.getGlobalOptimizer().optimize(globalPlan);
+
+      query = new Query(queryContext, queryId, querySubmitTime,
+          "", queryContext.getEventHandler(), masterPlan);
+
+      dispatcher.register(QueryEventType.class, query);
+
+      queryContext.getEventHandler().handle(new QueryEvent(queryId,
+          QueryEventType.INIT));
+      queryContext.getEventHandler().handle(new QueryEvent(queryId,
+          QueryEventType.START));
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      //TODO how set query failed(???)
+      //send FAIL query status
+      //this.statusMessage = StringUtils.stringifyException(e);
+    }
+  }
+
+  /**
+   * It initializes the final output and staging directory and sets
+   * them to variables.
+   */
+  private void initStagingDir() throws IOException {
+    String realUser;
+    String currentUser;
+    UserGroupInformation ugi;
+    ugi = UserGroupInformation.getLoginUser();
+    realUser = ugi.getShortUserName();
+    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+    String givenOutputTableName = queryConf.getOutputTable();
+    Path stagingDir;
+
+    // If final output directory is not given by an user,
+    // we use the query id as a output directory.
+    if (givenOutputTableName == null || givenOutputTableName.isEmpty()) {
+      this.isCreateTableStmt = false;
+      FileSystem defaultFS = FileSystem.get(queryConf);
+
+      Path homeDirectory = defaultFS.getHomeDirectory();
+      if (!defaultFS.exists(homeDirectory)) {
+        defaultFS.mkdirs(homeDirectory, new FsPermission(USER_DIR_PERMISSION));
+      }
+
+      Path userQueryDir = new Path(homeDirectory, TajoConstants.USER_QUERYDIR_PREFIX);
+
+      if (defaultFS.exists(userQueryDir)) {
+        FileStatus fsStatus = defaultFS.getFileStatus(userQueryDir);
+        String owner = fsStatus.getOwner();
+
+        if (!(owner.equals(currentUser) || owner.equals(realUser))) {
+          throw new IOException("The ownership on the user's query " +
+              "directory " + userQueryDir + " is not as expected. " +
+              "It is owned by " + owner + ". The directory must " +
+              "be owned by the submitter " + currentUser + " or " +
+              "by " + realUser);
+        }
+
+        if (!fsStatus.getPermission().equals(USER_DIR_PERMISSION)) {
+          LOG.info("Permissions on staging directory " + userQueryDir + " are " +
+              "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
+              "to correct value " + USER_DIR_PERMISSION);
+          defaultFS.setPermission(userQueryDir, new FsPermission(USER_DIR_PERMISSION));
+        }
+      } else {
+        defaultFS.mkdirs(userQueryDir,
+            new FsPermission(USER_DIR_PERMISSION));
+      }
+
+      stagingDir = StorageUtil.concatPath(userQueryDir, queryId.toString());
+
+      if (defaultFS.exists(stagingDir)) {
+        throw new IOException("The staging directory " + stagingDir
+            + "already exists. The directory must be unique to each query");
+      } else {
+        defaultFS.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
+      }
+
+      // Set the query id to the output table name
+      queryConf.setOutputTable(queryId.toString());
+
+    } else {
+      this.isCreateTableStmt = true;
+      Path warehouseDir = new Path(queryConf.getVar(TajoConf.ConfVars.ROOT_DIR),
+          TajoConstants.WAREHOUSE_DIR);
+      stagingDir = new Path(warehouseDir, queryConf.getOutputTable());
+
+      FileSystem fs = warehouseDir.getFileSystem(queryConf);
+      if (fs.exists(stagingDir)) {
+        throw new IOException("The staging directory " + stagingDir
+            + " already exists. The directory must be unique to each query");
+      } else {
+        // TODO - should have appropriate permission
+        fs.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
+      }
+    }
+
+    queryConf.setOutputPath(stagingDir);
+    outputPath = stagingDir;
+    LOG.info("Initialized Query Staging Dir: " + outputPath);
+  }
+
+  public Query getQuery() {
+    return query;
+  }
+
+  public void expiredSessionTimeout() {
+    stop();
+  }
+
+  public QueryContext getQueryContext() {
+    return queryContext;
+  }
+
+  public EventHandler getEventHandler() {
+    return queryContext.getEventHandler();
+  }
+
+  public void touchSessionTime() {
+    this.lastClientHeartbeat.set(System.currentTimeMillis());
+  }
+
+  public long getLastClientHeartbeat() {
+    return this.lastClientHeartbeat.get();
+  }
+
+  public QueryId getQueryId() {
+    return queryId;
+  }
+
+  public TajoProtos.QueryState getState() {
+    if(query == null) {
+      return TajoProtos.QueryState.QUERY_NOT_ASSIGNED;
+    } else {
+      return query.getState();
+    }
+  }
+
+  public class QueryContext {
+    EventHandler eventHandler;
+    public QueryMaster.QueryMasterContext getQueryMasterContext() {
+      return queryMasterContext;
+    }
+
+    public QueryConf getConf() {
+      return queryConf;
+    }
+
+    public Clock getClock() {
+      return queryMasterContext.getClock();
+    }
+
+    public Query getQuery() {
+      return query;
+    }
+
+    public QueryId getQueryId() {
+      return queryId;
+    }
+
+    public StorageManager getStorageManager() {
+      return queryMasterContext.getStorageManager();
+    }
+
+    public Path getOutputPath() {
+      return outputPath;
+    }
+
+    public boolean isCreateTableQuery() {
+      return isCreateTableStmt;
+    }
+
+    public synchronized EventHandler getEventHandler() {
+      if(eventHandler == null) {
+        eventHandler = dispatcher.getEventHandler();
+      }
+      return eventHandler;
+    }
+
+    public TajoAsyncDispatcher getDispatcher() {
+      return dispatcher;
+    }
+
+    public SubQuery getSubQuery(ExecutionBlockId id) {
+      return query.getSubQuery(id);
+    }
+
+    public Map<String, TableDesc> getTableDescMap() {
+      return tableDescMap;
+    }
+
+    public float getProgress() {
+      if(query == null) {
+        return 0.0f;
+      }
+      return query.getProgress();
+    }
+
+    public AbstractResourceAllocator getResourceAllocator() {
+      return (AbstractResourceAllocator)resourceAllocator;
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
index 69143c9..ec54244 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
@@ -30,11 +30,12 @@ import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.QueryUnitId;
 import org.apache.tajo.catalog.Schema;
 import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.ipc.QueryMasterProtocol.Partition;
 import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Partition;
 import org.apache.tajo.master.TaskState;
 import org.apache.tajo.master.event.*;
 import org.apache.tajo.storage.Fragment;
+import org.apache.tajo.util.TajoIdUtils;
 
 import java.net.URI;
 import java.net.URISyntaxException;
@@ -299,9 +300,8 @@ public class QueryUnit implements EventHandler<TaskEvent> {
 	}
 
   public QueryUnitAttempt newAttempt() {
-    QueryUnitAttempt attempt = new QueryUnitAttempt(
-        QueryIdFactory.newQueryUnitAttemptId(this.getId(),
-            ++lastAttemptId), this, eventHandler);
+    QueryUnitAttempt attempt = new QueryUnitAttempt(QueryIdFactory.newQueryUnitAttemptId(
+        this.getId(), ++lastAttemptId), this, eventHandler);
     return attempt;
   }
 
@@ -310,7 +310,7 @@ public class QueryUnit implements EventHandler<TaskEvent> {
   }
 
   public QueryUnitAttempt getAttempt(int attempt) {
-    return this.attempts.get(new QueryUnitAttemptId(this.getId(), attempt));
+    return this.attempts.get(QueryIdFactory.newQueryUnitAttemptId(this.getId(), attempt));
   }
 
   public QueryUnitAttempt getLastAttempt() {
@@ -433,7 +433,7 @@ public class QueryUnit implements EventHandler<TaskEvent> {
         stateMachine.doTransition(event.getType(), event);
       } catch (InvalidStateTransitonException e) {
         LOG.error("Can't handle this event at current state", e);
-        eventHandler.handle(new QueryEvent(getId().getQueryId(),
+        eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
             QueryEventType.INTERNAL_ERROR));
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
index add42aa..9b9c63f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
@@ -26,11 +26,12 @@ import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.TajoProtos.TaskAttemptState;
 import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.ipc.QueryMasterProtocol.Partition;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskCompletionReport;
-import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Partition;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
 import org.apache.tajo.master.event.*;
 import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.util.TajoIdUtils;
 
 import java.util.*;
 import java.util.concurrent.locks.Lock;
@@ -146,6 +147,10 @@ public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
     return this.hostName;
   }
 
+  public int getPort() {
+    return this.port;
+  }
+
   public void setHost(String host) {
     this.hostName = host;
   }
@@ -324,7 +329,7 @@ public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
       } catch (InvalidStateTransitonException e) {
         LOG.error("Can't handle this event at current state of "
             + event.getTaskAttemptId() + ")", e);
-        eventHandler.handle(new QueryEvent(getId().getQueryId(),
+        eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
             QueryEventType.INTERNAL_ERROR));
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
index cccfcba..a9b0e4b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -21,8 +21,8 @@ package org.apache.tajo.master.querymaster;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.catalog.*;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.statistics.TableStat;
@@ -70,7 +70,7 @@ public class Repartitioner {
     for (int i =0; i < 2; i++) {
       // TODO - temporarily tables should be stored in temporarily catalog for each query
       TableDesc tableDesc = subQuery.getContext().getTableDescMap().get(scans[i].getFromTable().getTableName());
-      if (scans[i].getTableId().startsWith(SubQueryId.PREFIX)) {
+      if (scans[i].getTableId().startsWith(ExecutionBlockId.EB_ID_PREFIX)) {
         tablePath = subQuery.getStorageManager().getTablePath(scans[i].getTableId());
       } else {
         tablePath = tableDesc.getPath();
@@ -355,12 +355,13 @@ public class Repartitioner {
   }
 
   public static String createBasicFetchUri(String hostName, int port,
-                                           SubQueryId childSid,
+                                           ExecutionBlockId childSid,
                                            int taskId, int attemptId) {
     String scheme = "http://";
     StringBuilder sb = new StringBuilder(scheme);
-    sb.append(hostName).append(":").append(port)
-        .append("/?").append("sid=").append(childSid.getId())
+    sb.append(hostName).append(":").append(port).append("/?")
+        .append("qid=").append(childSid.getQueryId().toString())
+        .append("&sid=").append(childSid.getId())
         .append("&").append("ta=").append(taskId).append("_").append(attemptId)
         .append("&").append("p=0")
         .append("&").append("type=r");
@@ -436,15 +437,16 @@ public class Repartitioner {
     return tasks;
   }
 
-  public static Collection<URI> createHashFetchURL(String hostAndPort, SubQueryId childSid,
+  public static Collection<URI> createHashFetchURL(String hostAndPort, ExecutionBlockId ebid,
                                        int partitionId, PartitionType type,
                                        List<IntermediateEntry> entries) {
     String scheme = "http://";
     StringBuilder urlPrefix = new StringBuilder(scheme);
-    urlPrefix.append(hostAndPort)
-        .append("/?").append("sid=").append(childSid.getId())
-        .append("&").append("p=").append(partitionId)
-        .append("&").append("type=");
+    urlPrefix.append(hostAndPort).append("/?")
+        .append("qid=").append(ebid.getQueryId().toString())
+        .append("&sid=").append(ebid.getId())
+        .append("&p=").append(partitionId)
+        .append("&type=");
     if (type == PartitionType.HASH) {
       urlPrefix.append("h");
     } else if (type == PartitionType.RANGE) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 94a6af8..fc60df7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.*;
 import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
@@ -52,7 +52,6 @@ import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
 import org.apache.tajo.master.TaskScheduler;
 import org.apache.tajo.master.TaskSchedulerImpl;
 import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
 import org.apache.tajo.storage.Fragment;
 import org.apache.tajo.storage.StorageManager;
 
@@ -79,7 +78,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
   private EventHandler eventHandler;
   private final StorageManager sm;
   private TaskSchedulerImpl taskScheduler;
-  private QueryContext context;
+  private QueryMasterTask.QueryContext context;
 
   private long startTime;
   private long finishTime;
@@ -87,7 +86,6 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
   volatile Map<QueryUnitId, QueryUnit> tasks = new ConcurrentHashMap<QueryUnitId, QueryUnit>();
   volatile Map<ContainerId, Container> containers = new ConcurrentHashMap<ContainerId, Container>();
 
-
   private static ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition();
   private StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent>
       stateMachine;
@@ -140,7 +138,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
   private int completedTaskCount = 0;
 
-  public SubQuery(QueryContext context, ExecutionBlock block, StorageManager sm) {
+  public SubQuery(QueryMasterTask.QueryContext context, ExecutionBlock block, StorageManager sm) {
     this.context = context;
     this.block = block;
     this.sm = sm;
@@ -152,7 +150,12 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     stateMachine = stateMachineFactory.make(this);
   }
 
-  public QueryContext getContext() {
+  public static boolean isRunningState(SubQueryState state) {
+    return state == SubQueryState.INIT || state == SubQueryState.NEW ||
+        state == SubQueryState.CONTAINER_ALLOCATED || state == SubQueryState.RUNNING;
+  }
+
+  public QueryMasterTask.QueryContext getContext() {
     return context;
   }
 
@@ -238,7 +241,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
     return context.getSubQuery(block.getChildBlock(scanForChild).getId());
   }
   
-  public SubQueryId getId() {
+  public ExecutionBlockId getId() {
     return block.getId();
   }
   
@@ -399,7 +402,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
   @Override
   public void handle(SubQueryEvent event) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType());
+      LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType() + ", preState=" + getState());
     }
 
     try {
@@ -420,13 +423,15 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
               + getState());
         }
       }
-    }
-
-    finally {
+    } finally {
       writeLock.unlock();
     }
   }
 
+  public void handleTaskRequestEvent(TaskRequestEvent event) {
+    taskScheduler.handleTaskRequestEvent(event);
+  }
+
   private static class InitAndRequestContainer implements MultipleArcTransition<SubQuery,
       SubQueryEvent, SubQueryState> {
 
@@ -564,7 +569,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
 
       } else { // Case 3: Others (Sort or Aggregation)
         int numTasks = getNonLeafTaskNum(subQuery);
-        SubQueryId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
+        ExecutionBlockId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
         SubQuery child = subQuery.context.getSubQuery(childId);
         tasks = Repartitioner.createNonLeafTask(subQuery, child, numTasks);
       }
@@ -594,7 +599,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       return maxTaskNum;
     }
 
-    public static long getInputVolume(QueryContext context, ExecutionBlock execBlock) {
+    public static long getInputVolume(QueryMasterTask.QueryContext context, ExecutionBlock execBlock) {
       Map<String, TableDesc> tableMap = context.getTableDescMap();
       if (execBlock.isLeafBlock()) {
         ScanNode outerScan = execBlock.getScanNodes()[0];
@@ -615,14 +620,13 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       ExecutionBlock execBlock = subQuery.getBlock();
       QueryUnit [] tasks = subQuery.getQueryUnits();
 
-      int numClusterNodes = subQuery.getContext().getNumClusterNode();
-      TajoConf conf =  subQuery.getContext().getConf();
-      int workerNum = conf.getIntVar(ConfVars.MAX_WORKER_PER_NODE);
-      int numRequest = Math.min(tasks.length, numClusterNodes * workerNum);
+      //TODO refresh worker's numClusterNodes
+      int numClusterNodes = subQuery.getContext().getResourceAllocator().getNumClusterNode();
+      int numRequest = numClusterNodes == 0 ? tasks.length: Math.min(tasks.length, numClusterNodes * 4);
 
       final Resource resource = Records.newRecord(Resource.class);
-      // TODO - for each different subquery, the volume of resource should be different.
-      resource.setMemory(2000);
+
+      resource.setMemory(512);
 
       Priority priority = Records.newRecord(Priority.class);
       priority.setPriority(subQuery.getPriority());
@@ -698,8 +702,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
           new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
               subQuery.getId(), allocationEvent.getAllocatedContainer()));
 
-      subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
-          SubQueryEventType.SQ_START));
+      subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_START));
     }
   }
 
@@ -734,7 +737,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       QueryUnitAttempt task = subQuery.getQueryUnit(taskEvent.getTaskId()).getSuccessfulAttempt();
 
       LOG.info(subQuery.getId() + " SubQuery Succeeded " + subQuery.completedTaskCount + "/"
-          + subQuery.tasks.size() + " on " + task.getHost());
+          + subQuery.tasks.size() + " on " + task.getHost() + ":" + task.getPort());
       if (subQuery.completedTaskCount == subQuery.tasks.size()) {
         subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
             SubQueryEventType.SQ_SUBQUERY_COMPLETED));
@@ -750,6 +753,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
       // TODO - Commit subQuery & do cleanup
       // TODO - records succeeded, failed, killed completed task
       // TODO - records metrics
+      LOG.info("SubQuery finished:" + subQuery.getId());
       subQuery.stopScheduler();
       subQuery.releaseContainers();
       subQuery.finish();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java
deleted file mode 100644
index c02ddaf..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.rm;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.AMResponse;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.client.AMRMClientImpl;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.master.event.ContainerAllocationEvent;
-import org.apache.tajo.master.event.ContainerAllocatorEventType;
-import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
-import org.apache.tajo.master.querymaster.SubQueryState;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class RMContainerAllocator extends AMRMClientImpl
-    implements EventHandler<ContainerAllocationEvent> {
-
-  /** Class Logger */
-  private static final Log LOG = LogFactory.getLog(RMContainerAllocator.
-      class.getName());
-
-  private QueryContext context;
-  private final EventHandler eventHandler;
-
-  public RMContainerAllocator(QueryContext context) {
-    super(context.getApplicationAttemptId());
-    this.context = context;
-    this.eventHandler = context.getDispatcher().getEventHandler();
-  }
-
-  public void init(Configuration conf) {
-    super.init(conf);
-  }
-
-  private static final int WAIT_INTERVAL_AVAILABLE_NODES = 500; // 0.5 second
-  public void start() {
-    super.start();
-
-    RegisterApplicationMasterResponse response;
-    try {
-      response = registerApplicationMaster("locahost", 10080, "http://localhost:1234");
-      context.setMaxContainerCapability(response.getMaximumResourceCapability().getMemory());
-      context.setMinContainerCapability(response.getMinimumResourceCapability().getMemory());
-
-      // If the number of cluster nodes is ZERO, it waits for available nodes.
-      AllocateResponse allocateResponse = allocate(0.0f);
-      while(allocateResponse.getNumClusterNodes() < 1) {
-        try {
-          Thread.sleep(WAIT_INTERVAL_AVAILABLE_NODES);
-          LOG.info("Waiting for Available Cluster Nodes");
-          allocateResponse = allocate(0);
-        } catch (InterruptedException e) {
-          LOG.error(e);
-        }
-      }
-      context.setNumClusterNodes(allocateResponse.getNumClusterNodes());
-    } catch (YarnRemoteException e) {
-      LOG.error(e);
-    }
-
-    startAllocatorThread();
-  }
-
-  protected Thread allocatorThread;
-  private final AtomicBoolean stopped = new AtomicBoolean(false);
-  private int rmPollInterval = 100;//millis
-  protected void startAllocatorThread() {
-    allocatorThread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
-          try {
-            try {
-              heartbeat();
-            } catch (YarnException e) {
-              LOG.error("Error communicating with RM: " + e.getMessage() , e);
-              return;
-            } catch (Exception e) {
-              LOG.error("ERROR IN CONTACTING RM. ", e);
-              // TODO: for other exceptions
-            }
-            Thread.sleep(rmPollInterval);
-          } catch (InterruptedException e) {
-            if (!stopped.get()) {
-              LOG.warn("Allocated thread interrupted. Returning.");
-            }
-            break;
-          }
-        }
-        LOG.info("Allocated thread stopped");
-      }
-    });
-    allocatorThread.setName("RMContainerAllocator");
-    allocatorThread.start();
-  }
-
-  public void stop() {
-    stopped.set(true);
-    allocatorThread.interrupt();
-    LOG.info("RMContainerAllocator stopped");
-    super.stop();
-  }
-
-  private final Map<Priority, SubQueryId> subQueryMap =
-      new HashMap<Priority, SubQueryId>();
-  private AtomicLong prevReportTime = new AtomicLong(0);
-  private int reportInterval = 5 * 1000; // second
-
-  public void heartbeat() throws Exception {
-    AllocateResponse allocateResponse = allocate(context.getProgress());
-    AMResponse response = allocateResponse.getAMResponse();
-    List<Container> allocatedContainers = response.getAllocatedContainers();
-
-    long currentTime = System.currentTimeMillis();
-    if((currentTime - prevReportTime.longValue()) >= reportInterval){
-      LOG.debug("Available Cluster Nodes: " + allocateResponse.getNumClusterNodes());
-      LOG.debug("Num of Allocated Containers: " + allocatedContainers.size());
-      LOG.info("Available Resource: " + response.getAvailableResources());
-      prevReportTime.set(currentTime);
-    }
-
-    if (allocatedContainers.size() > 0) {
-      LOG.info("================================================================");
-      for (Container container : response.getAllocatedContainers()) {
-        LOG.info("> Container Id: " + container.getId());
-        LOG.info("> Node Id: " + container.getNodeId());
-        LOG.info("> Resource (Mem): " + container.getResource().getMemory());
-        LOG.info("> State : " + container.getState());
-        LOG.info("> Priority: " + container.getPriority());
-      }
-      LOG.info("================================================================");
-
-      Map<SubQueryId, List<Container>> allocated = new HashMap<SubQueryId, List<Container>>();
-
-      for (Container container : allocatedContainers) {
-        SubQueryId subQueryId = subQueryMap.get(container.getPriority());
-        SubQueryState state = context.getSubQuery(subQueryId).getState();
-        if (!(isRunningState(state))) {
-          releaseAssignedContainer(container.getId());
-        } else {
-          if (allocated.containsKey(subQueryId)) {
-            allocated.get(subQueryId).add(container);
-          } else {
-            allocated.put(subQueryId, Lists.newArrayList(container));
-          }
-        }
-      }
-
-      for (Entry<SubQueryId, List<Container>> entry : allocated.entrySet()) {
-        eventHandler.handle(new SubQueryContainerAllocationEvent(entry.getKey(), entry.getValue()));
-      }
-    }
-  }
-
-  private static boolean isRunningState(SubQueryState state) {
-    return state == SubQueryState.INIT || state == SubQueryState.NEW ||
-        state == SubQueryState.CONTAINER_ALLOCATED || state == SubQueryState.RUNNING;
-  }
-
-  @Override
-  public void handle(ContainerAllocationEvent event) {
-
-    if (event.getType() == ContainerAllocatorEventType.CONTAINER_REQ) {
-      LOG.info(event);
-      subQueryMap.put(event.getPriority(), event.getSubQueryId());
-      addContainerRequest(new ContainerRequest(event.getCapability(), null, null,
-          event.getPriority(), event.getRequiredNum()));
-
-    } else if (event.getType() == ContainerAllocatorEventType.CONTAINER_DEALLOCATE) {
-      LOG.info(event);
-    } else {
-      LOG.info(event);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
new file mode 100644
index 0000000..9f7aeb1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import org.apache.hadoop.yarn.api.records.*;
+
+public class TajoWorkerContainer implements Container {
+  ContainerId id;
+  NodeId nodeId;
+  WorkerResource workerResource;
+
+  public WorkerResource getWorkerResource() {
+    return workerResource;
+  }
+
+  public void setWorkerResource(WorkerResource workerResource) {
+    this.workerResource = workerResource;
+  }
+
+  @Override
+  public ContainerId getId() {
+    return id;
+  }
+
+  @Override
+  public void setId(ContainerId id) {
+    this.id = id;
+  }
+
+  @Override
+  public NodeId getNodeId() {
+    return nodeId;
+  }
+
+  @Override
+  public void setNodeId(NodeId nodeId) {
+    this.nodeId = nodeId;
+  }
+
+  @Override
+  public String getNodeHttpAddress() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setNodeHttpAddress(String nodeHttpAddress) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public Resource getResource() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setResource(Resource resource) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public Priority getPriority() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setPriority(Priority priority) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public ContainerState getState() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setState(ContainerState state) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public ContainerToken getContainerToken() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setContainerToken(ContainerToken containerToken) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public ContainerStatus getContainerStatus() {
+    return null;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public void setContainerStatus(ContainerStatus containerStatus) {
+    //To change body of implemented methods use File | Settings | File Templates.
+  }
+
+  @Override
+  public int compareTo(Container container) {
+    return 0;  //To change body of implemented methods use File | Settings | File Templates.
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
new file mode 100644
index 0000000..f104637
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+public class TajoWorkerContainerId extends ContainerId {
+  ApplicationAttemptId applicationAttemptId;
+  int id;
+
+  @Override
+  public ApplicationAttemptId getApplicationAttemptId() {
+    return applicationAttemptId;
+  }
+
+  @Override
+  public void setApplicationAttemptId(ApplicationAttemptId atId) {
+    this.applicationAttemptId = atId;
+  }
+
+  @Override
+  public int getId() {
+    return id;
+  }
+
+  @Override
+  public void setId(int id) {
+    this.id = id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
new file mode 100644
index 0000000..be4b800
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -0,0 +1,394 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryJobEvent;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TajoWorkerResourceManager implements WorkerResourceManager {
+  private static final Log LOG = LogFactory.getLog(TajoWorkerResourceManager.class);
+
+  private TajoMaster.MasterContext masterContext;
+
+  private Map<String, WorkerResource> allWorkerResourceMap = new HashMap<String, WorkerResource>();
+  private Set<String> liveWorkerResources = new HashSet<String>();
+  private Set<String> deadWorkerResources = new HashSet<String>();
+
+  private Map<QueryId, WorkerResource> queryMasterMap = new HashMap<QueryId, WorkerResource>();
+
+  private Object workerResourceLock = new Object();
+
+  private final String queryIdSeed;
+
+  private WorkerResourceAllocationThread workerResourceAllocator;
+
+  private final BlockingQueue<WorkerResourceRequest> requestQueue;
+
+  private ReAllocationThread reAllocator;
+
+  private final List<WorkerResourceRequest> reAllocationList;
+
+  private AtomicBoolean stopped = new AtomicBoolean(false);
+
+  private int queryMasterMemoryMB;
+
+  public TajoWorkerResourceManager(TajoMaster.MasterContext masterContext) {
+    this.masterContext = masterContext;
+    this.queryIdSeed = String.valueOf(System.currentTimeMillis());
+    this.queryMasterMemoryMB = masterContext.getConf().getInt("tajo.querymaster.memoryMB", 512);
+
+    requestQueue = new LinkedBlockingDeque<WorkerResourceRequest>();
+    reAllocationList = new ArrayList<WorkerResourceRequest>();
+
+    workerResourceAllocator = new WorkerResourceAllocationThread();
+    workerResourceAllocator.start();
+
+    reAllocator = new ReAllocationThread();
+    reAllocator.start();
+  }
+
+  @Override
+  public void stop() {
+    if(stopped.get()) {
+      return;
+    }
+    stopped.set(true);
+    if(workerResourceAllocator != null) {
+      workerResourceAllocator.interrupt();
+    }
+
+    if(reAllocator != null) {
+      reAllocator.interrupt();
+    }
+  }
+
+  @Override
+  public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress) {
+    List<WorkerResource> workerResources = chooseWorkers(true, 1, 1, 1);
+    if(workerResources.size() == 0) {
+      //TODO if resource available, assign worker.
+      LOG.warn("No available resource for querymaster:" + queryInProgress.getQueryId());
+      return null;
+    }
+    WorkerResource queryMasterWorker = workerResources.get(0);
+    synchronized(workerResourceLock) {
+      queryMasterMap.put(queryInProgress.getQueryId(), queryMasterWorker);
+    }
+    LOG.info(queryInProgress.getQueryId() + "'s QueryMaster is " + queryMasterWorker);
+    return queryMasterWorker;
+  }
+
+  @Override
+  public void startQueryMaster(QueryInProgress queryInProgress) {
+    WorkerResource queryMasterWorkerResource = null;
+    synchronized(workerResourceLock) {
+      queryMasterWorkerResource = queryMasterMap.get(queryInProgress.getQueryId());
+    }
+
+    if(queryMasterWorkerResource != null) {
+      startQueryMaster(queryInProgress.getQueryId(), queryMasterWorkerResource);
+    } else {
+      //add queue
+      TajoMasterProtocol.WorkerResourceAllocationRequest request =
+          TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+            .setMemoryMBSlots(queryMasterMemoryMB)
+            .setDiskSlots(1)
+            .setExecutionBlockId(QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0).getProto())
+            .setNumWorks(1)
+            .build();
+      try {
+        requestQueue.put(new WorkerResourceRequest(queryInProgress.getQueryId(), true, request, null));
+      } catch (InterruptedException e) {
+      }
+    }
+  }
+
+  private void startQueryMaster(QueryId queryId, WorkerResource workResource) {
+    QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+    if(queryInProgress == null) {
+      LOG.warn("No QueryInProgress while starting  QueryMaster:" + queryId);
+      return;
+    }
+    queryInProgress.getQueryInfo().setQueryMasterResource(workResource);
+
+    //fire QueryJobStart event
+    queryInProgress.getEventHandler().handle(
+        new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
+  }
+
+  @Override
+  public String getSeedQueryId() throws IOException {
+    return queryIdSeed;
+  }
+
+  @Override
+  public void allocateWorkerResources(
+      TajoMasterProtocol.WorkerResourceAllocationRequest request,
+      RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack) {
+    try {
+      //TODO checking queue size
+      requestQueue.put(new WorkerResourceRequest(
+          new QueryId(request.getExecutionBlockId().getQueryId()), false, request, callBack));
+    } catch (InterruptedException e) {
+      LOG.error(e.getMessage(), e);
+    }
+  }
+
+  class WorkerResourceRequest {
+    boolean queryMasterRequest;
+    QueryId queryId;
+    TajoMasterProtocol.WorkerResourceAllocationRequest request;
+    RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack;
+    WorkerResourceRequest(
+        QueryId queryId,
+        boolean queryMasterRequest, TajoMasterProtocol.WorkerResourceAllocationRequest request,
+        RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack) {
+      this.queryId = queryId;
+      this.queryMasterRequest = queryMasterRequest;
+      this.request = request;
+      this.callBack = callBack;
+    }
+  }
+
+  class WorkerResourceAllocationThread extends Thread {
+    @Override
+    public void run() {
+      LOG.info("====>WorkerResourceAllocationThread start");
+      while(!stopped.get()) {
+        try {
+          WorkerResourceRequest resourceRequest = requestQueue.take();
+          List<WorkerResource> workerResources = chooseWorkers(false,
+              resourceRequest.request.getMemoryMBSlots(),
+              resourceRequest.request.getDiskSlots(),
+              resourceRequest.request.getNumWorks());
+
+          LOG.info("====> allocateWorkerResources:" +
+              (new ExecutionBlockId(resourceRequest.request.getExecutionBlockId())) +
+              ", required:" + resourceRequest.request.getNumWorks() + ", allocated:" + workerResources.size() +
+              ", queryMasterRequest=" + resourceRequest.queryMasterRequest +
+              ", liveWorkers=" + liveWorkerResources.size());
+//          if(LOG.isDebugEnabled()) {
+//            LOG.debug("====> allocateWorkerResources:" +
+//                (new ExecutionBlockId(resourceRequest.request.getExecutionBlockId())) +
+//                ", required:" + resourceRequest.request.getNumWorks() + ", allocated:" + workerResources.size());
+//          } else {
+//            LOG.info("====> allocateWorkerResources: required:" + resourceRequest.request.getNumWorks() +
+//                ", allocated:" + workerResources.size() + ", queryMasterRequest=" + resourceRequest.queryMasterRequest);
+//          }
+
+          if(workerResources.size() > 0) {
+            if(resourceRequest.queryMasterRequest) {
+              startQueryMaster(resourceRequest.queryId, workerResources.get(0));
+            } else {
+              List<String> workerHosts = new ArrayList<String>();
+
+              for(WorkerResource eachWorker: workerResources) {
+                workerHosts.add(eachWorker.getAllocatedHost() + ":" + eachWorker.getPorts()[0]);
+              }
+              resourceRequest.callBack.run(TajoMasterProtocol.WorkerResourceAllocationResponse.newBuilder()
+                  .setExecutionBlockId(resourceRequest.request.getExecutionBlockId())
+                  .addAllAllocatedWorks(workerHosts)
+                  .build()
+              );
+            }
+          } else {
+            if(LOG.isDebugEnabled()) {
+              LOG.debug("=========================================");
+              LOG.debug("Available Workers");
+              for(String liveWorker: liveWorkerResources) {
+                LOG.debug(allWorkerResourceMap.get(liveWorker).toString());
+              }
+              LOG.debug("=========================================");
+            }
+          }
+          if(workerResources.size() < resourceRequest.request.getNumWorks()) {
+            reAllocationList.add(new WorkerResourceRequest(
+                resourceRequest.queryId,
+                resourceRequest.queryMasterRequest,
+                TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+                  .setMemoryMBSlots(resourceRequest.request.getMemoryMBSlots())
+                  .setDiskSlots(resourceRequest.request.getDiskSlots())
+                  .setExecutionBlockId(resourceRequest.request.getExecutionBlockId())
+                  .setNumWorks(resourceRequest.request.getNumWorks() - workerResources.size())
+                  .build(),
+                resourceRequest.callBack));
+          }
+        } catch(InterruptedException ie) {
+        }
+      }
+    }
+  }
+
+  class ReAllocationThread extends Thread {
+    public void run() {
+      List<WorkerResourceRequest> copiedList = new ArrayList<WorkerResourceRequest>();
+      while(!stopped.get()) {
+        copiedList.clear();
+        synchronized(reAllocationList) {
+          try {
+            reAllocationList.wait(3 * 1000);
+          } catch (InterruptedException e) {
+            if(stopped.get()) {
+              break;
+            }
+          }
+          copiedList.addAll(reAllocationList);
+        }
+
+        for(WorkerResourceRequest eachRequest: copiedList) {
+          try {
+            requestQueue.put(eachRequest);
+          } catch (InterruptedException e) {
+            break;
+          }
+        }
+        synchronized(reAllocationList) {
+          reAllocationList.clear();
+        }
+      }
+    }
+  }
+
+  private List<WorkerResource> chooseWorkers(boolean queryMaster,
+                                             int requiredMemoryMBSlots, int requiredDiskSlots,
+                                             int numWorkers) {
+    List<WorkerResource> selectedWorkers = new ArrayList<WorkerResource>();
+
+    int selectedCount = 0;
+
+    synchronized(workerResourceLock) {
+      for(String eachWorker: liveWorkerResources) {
+        if(selectedCount >= numWorkers) {
+          break;
+        }
+        WorkerResource workerResource = allWorkerResourceMap.get(eachWorker);
+        if(workerResource.getAvailableMemoryMBSlots() >= requiredMemoryMBSlots &&
+            workerResource.getAvailableDiskSlots() >= requiredDiskSlots) {
+          if(queryMaster && workerResource.isQueryMasterAllocated()) {
+            continue;
+          }
+          workerResource.addUsedMemoryMBSlots(requiredMemoryMBSlots);
+          workerResource.addUsedDiskSlots(requiredDiskSlots);
+          workerResource.setQueryMasterAllocated(queryMaster);
+          selectedWorkers.add(workerResource);
+          selectedCount++;
+        }
+      }
+    }
+
+    return selectedWorkers;
+  }
+
+  public Collection<WorkerResource> getClusterWorkResources() {
+    return Collections.unmodifiableCollection(allWorkerResourceMap.values());
+  }
+
+  @Override
+  public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource) {
+    synchronized(workerResourceLock) {
+      WorkerResource managedWorkerResource = allWorkerResourceMap.get(workerResource.getId());
+      if(managedWorkerResource != null) {
+        managedWorkerResource.releaseResource(workerResource);
+      }
+    }
+
+    synchronized(reAllocationList) {
+      reAllocationList.notifyAll();
+    }
+  }
+
+  @Override
+  public boolean isQueryMasterStopped(QueryId queryId) {
+    synchronized(workerResourceLock) {
+      return !queryMasterMap.containsKey(queryId);
+    }
+  }
+
+  @Override
+  public void init(Configuration conf) {
+  }
+
+  @Override
+  public void stopQueryMaster(QueryId queryId) {
+    WorkerResource workerResource = null;
+    synchronized(workerResourceLock) {
+      workerResource = queryMasterMap.remove(queryId);
+    }
+    LOG.info("release QueryMaster resource:" + queryId + "," + workerResource.isQueryMasterAllocated());
+    if(workerResource != null) {
+      releaseWorkerResource(queryId, workerResource);
+    }
+  }
+
+  public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request) {
+    synchronized(workerResourceLock) {
+      String hostAndPort = request.getTajoWorkerHost() + ":" + request.getTajoWorkerPort();
+      if(allWorkerResourceMap.containsKey(hostAndPort)) {
+        if(deadWorkerResources.contains(hostAndPort)) {
+          deadWorkerResources.remove(hostAndPort);
+          liveWorkerResources.add(hostAndPort);
+        }
+        WorkerResource workerResource = allWorkerResourceMap.get(hostAndPort);
+        workerResource.setLastHeartbeat(System.currentTimeMillis());
+        workerResource.setWorkerStatus(WorkerStatus.LIVE);
+      } else {
+        WorkerResource workerResource = new WorkerResource();
+        workerResource.setAllocatedHost(request.getTajoWorkerHost());
+
+        int[] ports = new int[] { request.getTajoWorkerPort(), request.getTajoWorkerClientPort() };
+
+        workerResource.setPorts(ports);
+
+        workerResource.setLastHeartbeat(System.currentTimeMillis());
+        workerResource.setWorkerStatus(WorkerStatus.LIVE);
+        if(request.getServerStatus() != null) {
+          workerResource.setMemoryMBSlots(request.getServerStatus().getSystem().getTotalMemoryMB());
+          workerResource.setCpuCoreSlots(request.getServerStatus().getSystem().getAvailableProcessors());
+          workerResource.setDiskSlots(request.getServerStatus().getDiskSlots());
+        } else {
+          workerResource.setMemoryMBSlots(4096);
+          workerResource.setDiskSlots(4);
+          workerResource.setCpuCoreSlots(4);
+        }
+
+        allWorkerResourceMap.put(workerResource.getId(), workerResource);
+        liveWorkerResources.add(hostAndPort);
+
+        LOG.info("====> TajoWorker:" + workerResource + " added in live TajoWorker list");
+
+        workerResourceLock.notifyAll();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
new file mode 100644
index 0000000..00bd509
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class WorkerResource {
+  private static final Log LOG = LogFactory.getLog(WorkerResource.class);
+
+  private String allocatedHost;
+  private int[] ports;
+
+  private int diskSlots;
+  private int cpuCoreSlots;
+  private int memoryMBSlots;
+
+  private int usedDiskSlots;
+  private int usedMemoryMBSlots;
+  private int usedCpuCoreSlots;
+
+  private boolean queryMasterAllocated;
+
+  private WorkerStatus workerStatus;
+
+  private long lastHeartbeat;
+
+  public String getId() {
+    if(ports.length > 0) {
+      return allocatedHost + ":" + ports[0];
+    } else {
+      return allocatedHost;
+    }
+  }
+
+  public String getAllocatedHost() {
+    return allocatedHost;
+  }
+
+  public void setAllocatedHost(String allocatedHost) {
+    this.allocatedHost = allocatedHost;
+  }
+
+  public int[] getPorts() {
+    return ports;
+  }
+
+  public void setPorts(int[] ports) {
+    this.ports = ports;
+  }
+
+  public void addUsedDiskSlots(int diskSlots) {
+    usedDiskSlots += diskSlots;
+  }
+
+  public void addUsedMemoryMBSlots(int memoryMBSlots) {
+    usedMemoryMBSlots += memoryMBSlots;
+  }
+
+  public void addUsedCpuCoreSlots(int cpuCoreSlots) {
+    usedCpuCoreSlots += cpuCoreSlots;
+  }
+
+  public int getDiskSlots() {
+    return diskSlots;
+  }
+
+  public void setDiskSlots(int diskSlots) {
+    this.diskSlots = diskSlots;
+  }
+
+  public int getCpuCoreSlots() {
+    return cpuCoreSlots;
+  }
+
+  public void setCpuCoreSlots(int cpuCoreSlots) {
+    this.cpuCoreSlots = cpuCoreSlots;
+  }
+
+  public int getMemoryMBSlots() {
+    return memoryMBSlots;
+  }
+
+  public void setMemoryMBSlots(int memoryMBSlots) {
+    this.memoryMBSlots = memoryMBSlots;
+  }
+
+  public int getAvailableDiskSlots() {
+    return diskSlots - usedDiskSlots;
+  }
+
+  public int getAvailableMemoryMBSlots() {
+    return memoryMBSlots - usedMemoryMBSlots;
+  }
+
+  @Override
+  public String toString() {
+    return "host:" + allocatedHost + ", port=" + portsToStr() + ", slots=" + memoryMBSlots + ":" + cpuCoreSlots + ":" + diskSlots +
+        ", used=" + usedMemoryMBSlots + ":" + usedCpuCoreSlots + ":" + usedDiskSlots;
+  }
+
+  private String portsToStr() {
+    if(ports == null) {
+      return "null";
+    }
+    String result = "";
+    String prefix = "";
+    for(int i = 0; i < ports.length; i++) {
+      result += prefix + ports[i];
+      prefix = ",";
+    }
+
+    return result;
+  }
+
+  public void setLastHeartbeat(long heartbeatTime) {
+    this.lastHeartbeat = heartbeatTime;
+  }
+
+  public int getUsedMemoryMBSlots() {
+    return usedMemoryMBSlots;
+  }
+
+  public void setUsedMemoryMBSlots(int usedMemoryMBSlots) {
+    this.usedMemoryMBSlots = usedMemoryMBSlots;
+  }
+
+  public int getUsedCpuCoreSlots() {
+    return usedCpuCoreSlots;
+  }
+
+  public void setUsedCpuCoreSlots(int usedCpuCoreSlots) {
+    this.usedCpuCoreSlots = usedCpuCoreSlots;
+  }
+
+  public int getUsedDiskSlots() {
+    return usedDiskSlots;
+  }
+
+  public void setUsedDiskSlots(int usedDiskSlots) {
+    this.usedDiskSlots = usedDiskSlots;
+  }
+
+  public WorkerStatus getWorkerStatus() {
+    return workerStatus;
+  }
+
+  public void setWorkerStatus(WorkerStatus workerStatus) {
+    this.workerStatus = workerStatus;
+  }
+
+  public long getLastHeartbeat() {
+    return lastHeartbeat;
+  }
+
+  public boolean isQueryMasterAllocated() {
+    return queryMasterAllocated;
+  }
+
+  public void setQueryMasterAllocated(boolean queryMasterAllocated) {
+    this.queryMasterAllocated = queryMasterAllocated;
+  }
+
+  public void releaseResource(WorkerResource workerResource) {
+    if(workerResource.isQueryMasterAllocated()) {
+        queryMasterAllocated = false;
+    }
+
+    usedMemoryMBSlots -= workerResource.memoryMBSlots;
+    usedDiskSlots -= workerResource.diskSlots;
+
+    if(usedMemoryMBSlots < 0 || usedDiskSlots < 0 || usedCpuCoreSlots < 0) {
+//      LOG.warn("Used resources can't be a minus.");
+      LOG.trace("Used resources can't be a minus.");
+      LOG.warn(this + " ==> " + workerResource);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
new file mode 100644
index 0000000..a066321
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+
+import java.io.IOException;
+
+public interface WorkerResourceManager {
+
+  /**
+   * select Worker for QueryMaster
+   * @param queryInProgress
+   * @return
+   */
+  public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress);
+
+  public void allocateWorkerResources(TajoMasterProtocol.WorkerResourceAllocationRequest request,
+      RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> rpcCallBack);
+
+  /**
+   * start Worker for query master(YARN) or assign query master role(Standby Mode)
+   * @param queryInProgress
+   */
+  public void startQueryMaster(QueryInProgress queryInProgress);
+
+  public String getSeedQueryId() throws IOException;
+
+  public boolean isQueryMasterStopped(QueryId queryId);
+
+  public void init(Configuration conf);
+
+  public void stopQueryMaster(QueryId queryId);
+
+  public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request);
+
+  public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource);
+
+  public void stop();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java
new file mode 100644
index 0000000..21ad7d7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+public enum WorkerStatus {
+  LIVE,
+  DEAD,
+  DECOMMISSION
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
new file mode 100644
index 0000000..d7adad8
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.AMResponse;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.AMRMClientImpl;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.master.event.ContainerAllocationEvent;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
+import org.apache.tajo.master.querymaster.Query;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class YarnRMContainerAllocator extends AMRMClientImpl
+    implements EventHandler<ContainerAllocationEvent> {
+
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(YarnRMContainerAllocator.
+      class.getName());
+
+  private QueryMasterTask.QueryContext context;
+  private final EventHandler eventHandler;
+
+  public YarnRMContainerAllocator(QueryMasterTask.QueryContext context) {
+    super(ApplicationIdUtils.createApplicationAttemptId(context.getQueryId()));
+    this.context = context;
+    this.eventHandler = context.getDispatcher().getEventHandler();
+  }
+
+  public void init(Configuration conf) {
+    super.init(conf);
+  }
+
+  private static final int WAIT_INTERVAL_AVAILABLE_NODES = 500; // 0.5 second
+  public void start() {
+    super.start();
+
+    RegisterApplicationMasterResponse response;
+    try {
+      response = registerApplicationMaster("localhost", 10080, "http://localhost:1234");
+      context.getResourceAllocator().setMaxContainerCapability(response.getMaximumResourceCapability().getMemory());
+      context.getResourceAllocator().setMinContainerCapability(response.getMinimumResourceCapability().getMemory());
+
+      // If the number of cluster nodes is ZERO, it waits for available nodes.
+      AllocateResponse allocateResponse = allocate(0.0f);
+      while(allocateResponse.getNumClusterNodes() < 1) {
+        try {
+          Thread.sleep(WAIT_INTERVAL_AVAILABLE_NODES);
+          LOG.info("Waiting for Available Cluster Nodes");
+          allocateResponse = allocate(0);
+        } catch (InterruptedException e) {
+          LOG.error(e);
+        }
+      }
+      context.getResourceAllocator().setNumClusterNodes(allocateResponse.getNumClusterNodes());
+    } catch (YarnRemoteException e) {
+      LOG.error(e);
+    }
+
+    startAllocatorThread();
+  }
+
+  protected Thread allocatorThread;
+  private final AtomicBoolean stopped = new AtomicBoolean(false);
+  private int rmPollInterval = 1000;//millis
+
+  protected void startAllocatorThread() {
+    allocatorThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+          try {
+            try {
+              heartbeat();
+            } catch (YarnException e) {
+              LOG.error("Error communicating with RM: " + e.getMessage() , e);
+              return;
+            } catch (Exception e) {
+              LOG.error("ERROR IN CONTACTING RM. ", e);
+              // TODO: for other exceptions
+              if(stopped.get()) {
+                break;
+              }
+            }
+            Thread.sleep(rmPollInterval);
+          } catch (InterruptedException e) {
+            if (!stopped.get()) {
+              LOG.warn("Allocated thread interrupted. Returning.");
+            }
+            break;
+          }
+        }
+        LOG.info("Allocated thread stopped");
+      }
+    });
+    allocatorThread.setName("YarnRMContainerAllocator");
+    allocatorThread.start();
+  }
+
+  public void stop() {
+    if(stopped.get()) {
+      return;
+    }
+    LOG.info("un-registering ApplicationMaster(QueryMaster):" + appAttemptId);
+    stopped.set(true);
+
+    try {
+      FinalApplicationStatus status = FinalApplicationStatus.UNDEFINED;
+      Query query = context.getQuery();
+      if (query != null) {
+        TajoProtos.QueryState state = query.getState();
+        if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+          status = FinalApplicationStatus.SUCCEEDED;
+        } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
+          status = FinalApplicationStatus.FAILED;
+        } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
+          status = FinalApplicationStatus.FAILED;
+        }
+      }
+      unregisterApplicationMaster(status, "tajo query finished", null);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    allocatorThread.interrupt();
+    LOG.info("un-registered ApplicationMAster(QueryMaster) stopped:" + appAttemptId);
+
+    super.stop();
+  }
+
+  private final Map<Priority, ExecutionBlockId> subQueryMap =
+      new HashMap<Priority, ExecutionBlockId>();
+
+  public void heartbeat() throws Exception {
+    AllocateResponse allocateResponse = allocate(context.getProgress());
+    AMResponse response = allocateResponse.getAMResponse();
+    if(response == null) {
+      LOG.warn("AM Response is null");
+      return;
+    }
+    List<Container> allocatedContainers = response.getAllocatedContainers();
+
+    LOG.info("Available Cluster Nodes: " + allocateResponse.getNumClusterNodes());
+    LOG.info("Available Resource: " + response.getAvailableResources());
+    LOG.info("Num of Allocated Containers: " + response.getAllocatedContainers().size());
+    if (response.getAllocatedContainers().size() > 0) {
+      LOG.info("================================================================");
+      for (Container container : response.getAllocatedContainers()) {
+        LOG.info("> Container Id: " + container.getId());
+        LOG.info("> Node Id: " + container.getNodeId());
+        LOG.info("> Resource (Mem): " + container.getResource().getMemory());
+        LOG.info("> State : " + container.getState());
+        LOG.info("> Priority: " + container.getPriority());
+      }
+      LOG.info("================================================================");
+    }
+
+    Map<ExecutionBlockId, List<Container>> allocated = new HashMap<ExecutionBlockId, List<Container>>();
+    if (allocatedContainers.size() > 0) {
+      for (Container container : allocatedContainers) {
+        ExecutionBlockId executionBlockId = subQueryMap.get(container.getPriority());
+        SubQueryState state = context.getSubQuery(executionBlockId).getState();
+        if (!(SubQuery.isRunningState(state) && subQueryMap.containsKey(container.getPriority()))) {
+          releaseAssignedContainer(container.getId());
+          synchronized (subQueryMap) {
+            subQueryMap.remove(container.getPriority());
+          }
+        } else {
+          if (allocated.containsKey(executionBlockId)) {
+            allocated.get(executionBlockId).add(container);
+          } else {
+            allocated.put(executionBlockId, Lists.newArrayList(container));
+          }
+        }
+      }
+
+      for (Entry<ExecutionBlockId, List<Container>> entry : allocated.entrySet()) {
+        eventHandler.handle(new SubQueryContainerAllocationEvent(entry.getKey(), entry.getValue()));
+      }
+    }
+  }
+
+  @Override
+  public void handle(ContainerAllocationEvent event) {
+
+    if (event.getType() == ContainerAllocatorEventType.CONTAINER_REQ) {
+      LOG.info(event);
+      subQueryMap.put(event.getPriority(), event.getExecutionBlockId());
+      addContainerRequest(new ContainerRequest(event.getCapability(), null, null,
+          event.getPriority(), event.getRequiredNum()));
+
+    } else if (event.getType() == ContainerAllocatorEventType.CONTAINER_DEALLOCATE) {
+      LOG.info(event);
+    } else {
+      LOG.info(event);
+    }
+  }
+}