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/14 08:47:59 UTC

[1/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Updated Branches:
  refs/heads/master a2693722b -> 9d0208839


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 9843762..9d63317 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
@@ -20,9 +20,6 @@ package org.apache.tajo.rpc;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configuration.IntegerRanges;
-import org.apache.hadoop.net.NetUtils;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.channel.Channel;
 import org.jboss.netty.channel.ChannelFactory;
@@ -30,7 +27,9 @@ import org.jboss.netty.channel.ChannelPipelineFactory;
 import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
 
 import java.io.IOException;
-import java.net.*;
+import java.net.DatagramSocket;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
 import java.util.Random;
 import java.util.concurrent.Executors;
 
@@ -42,7 +41,7 @@ public class NettyServerBase {
   protected ChannelFactory factory;
   protected ChannelPipelineFactory pipelineFactory;
   protected ServerBootstrap bootstrap;
-  private Channel channel;
+  protected Channel channel;
 
   public NettyServerBase(InetSocketAddress addr) {
     if (addr.getPort() == 0) {
@@ -81,39 +80,8 @@ public class NettyServerBase {
   public void start() {
     this.channel = bootstrap.bind(serverAddr);
     this.bindAddress = (InetSocketAddress) channel.getLocalAddress();
-    LOG.info("RpcServer on " + this.bindAddress);
-  }
 
-  public static void bind(ServerSocket socket, InetSocketAddress address,
-                          int backlog, Configuration conf, String rangeConf) throws IOException {
-    try {
-      IntegerRanges range = null;
-      if (rangeConf != null) {
-        range = conf.getRange(rangeConf, "");
-      }
-      if (range == null || range.isEmpty() || (address.getPort() != 0)) {
-        socket.bind(address, backlog);
-      } else {
-        for (Integer port : range) {
-          if (socket.isBound()) break;
-          try {
-            InetSocketAddress temp = new InetSocketAddress(address.getAddress(),
-                port);
-            socket.bind(temp, backlog);
-          } catch(BindException e) {
-            //Ignored
-          }
-        }
-        if (!socket.isBound()) {
-          throw new BindException("Could not find a free port in "+range);
-        }
-      }
-    } catch (SocketException e) {
-      throw NetUtils.wrapException(null,
-          0,
-          address.getHostName(),
-          address.getPort(), e);
-    }
+    LOG.info("RpcServer on " + this.bindAddress);
   }
 
   public Channel getChannel() {
@@ -121,16 +89,22 @@ public class NettyServerBase {
   }
 
   public void shutdown() {
-    channel.close().awaitUninterruptibly();
+    if(channel != null) {
+      channel.close().awaitUninterruptibly();
+    }
+    if(factory != null) {
+      factory.releaseExternalResources();
+    }
     LOG.info("RpcServer (" + org.apache.tajo.util.NetUtils.getIpPortString(bindAddress)
         + ") shutdown");
   }
 
   private static final Random randomPort = new Random(System.currentTimeMillis());
-  private static int getUnusedPort() throws IOException {
+  private synchronized static int getUnusedPort() throws IOException {
     while (true) {
       int port = randomPort.nextInt(10000) + 50000;
       if (available(port)) {
+        LOG.info("Found unused port:" + port);
         return port;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcServer.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcServer.java
index ca70536..d4ea8d4 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcServer.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcServer.java
@@ -63,7 +63,6 @@ public class ProtoBlockingRpcServer extends NettyServerBase {
     @Override
     public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
         throws Exception {
-
       final RpcRequest request = (RpcRequest) e.getMessage();
 
       String methodName = request.getMethodName();
@@ -74,7 +73,6 @@ public class ProtoBlockingRpcServer extends NettyServerBase {
         throw new RemoteCallException(request.getId(),
             new NoSuchMethodException(methodName));
       }
-
       Message paramProto = null;
       if (request.hasRequestMessage()) {
         try {
@@ -86,7 +84,6 @@ public class ProtoBlockingRpcServer extends NettyServerBase {
           throw new RemoteCallException(request.getId(), methodDescriptor, t);
         }
       }
-
       Message returnValue;
       RpcController controller = new NettyRpcController();
 
@@ -107,7 +104,6 @@ public class ProtoBlockingRpcServer extends NettyServerBase {
       if (controller.failed()) {
         builder.setErrorMessage(controller.errorText());
       }
-
       e.getChannel().write(builder.build());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyAsyncRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyAsyncRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyAsyncRpc.java
deleted file mode 100644
index d05eef6..0000000
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyAsyncRpc.java
+++ /dev/null
@@ -1,115 +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.rpc;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
-import org.apache.tajo.rpc.test.DummyProtos.MulRequest1;
-import org.apache.tajo.rpc.test.DummyProtos.MulResponse;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestNettyAsyncRpc {
-  public static String MESSAGE = TestNettyAsyncRpc.class.getName();
-  NettyRpcServer server;
-  DummyClientInterface proxy;
-
-  public static interface DummyServerInterface {
-    public Object throwException(MulRequest1 request) throws IOException;
-
-    public MulResponse mul(MulRequest1 req1);
-
-    public void nullParameterTest(NullProto proto);
-  }
-
-  public static interface DummyClientInterface {
-    public void throwException(Callback<Object> callback, MulRequest1 request)
-        throws IOException;
-
-    public void mul(Callback<MulResponse> callback, MulRequest1 req1);
-
-    public void nullParameterTest(Callback<Object> callback, NullProto proto);
-  }
-
-  public static class DummyServer implements DummyServerInterface {
-    @Override
-    public Object throwException(MulRequest1 request) throws IOException {
-      throw new IOException();
-    }
-
-    @Override
-    public MulResponse mul(MulRequest1 req1) {
-      int x1_1 = req1.getX1();
-      int x1_2 = req1.getX2();
-
-      int result1 = x1_1 * x1_2;
-
-      MulResponse rst = MulResponse.newBuilder().setResult1(result1)
-          .setResult2(400).build();
-      return rst;
-    }
-
-    public void nullParameterTest(NullProto proto) {
-    }
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    server = NettyRpc.getProtoParamRpcServer(new DummyServer(),
-        DummyServerInterface.class, new InetSocketAddress(0));
-    server.start();
-
-    InetSocketAddress addr = server.getBindAddress();
-
-    proxy = (DummyClientInterface) NettyRpc.getProtoParamAsyncRpcProxy(
-        DummyServerInterface.class, DummyClientInterface.class, addr);
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    server.shutdown();
-  }
-
-  MulResponse answer1 = null;
-
-  @Test
-  public void testRpcProtoType() throws Exception {
-    MulRequest1 req1 = MulRequest1.newBuilder().setX1(10).setX2(20).build();
-
-    Callback<MulResponse> cb = new Callback<MulResponse>();
-    proxy.mul(cb, req1);
-
-    MulResponse resp = (MulResponse) cb.get();
-    assertEquals(200, resp.getResult1());
-    assertEquals(400, resp.getResult2());
-  }
-
-  @Test
-  public void testNullParameter() throws Exception {
-    NullProto np = NullProto.newBuilder().build();
-
-    Callback<Object> cb = new Callback<Object>();
-    proxy.nullParameterTest(cb, np);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyBlockingRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyBlockingRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyBlockingRpc.java
deleted file mode 100644
index 5967dfa..0000000
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestNettyBlockingRpc.java
+++ /dev/null
@@ -1,110 +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.rpc;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.NullProto;
-import org.apache.tajo.rpc.test.DummyProtos.MulRequest1;
-import org.apache.tajo.rpc.test.DummyProtos.MulResponse;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestNettyBlockingRpc {
-
-  public static String MESSAGE = TestNettyBlockingRpc.class.getName();
-  NettyRpcServer server;
-  DummyClientInterface proxy;
-
-  // !. Write Interface and implement class according to communication way
-  public static interface DummyServerInterface {
-    public MulResponse mul(MulRequest1 req1);
-
-    public void nullParameterTest(NullProto proto);
-  }
-
-  public static interface DummyClientInterface {
-    public void throwException(MulRequest1 request) throws RemoteException;
-
-    public MulResponse mul(MulRequest1 req1) throws RemoteException;
-
-    public void nullParameterTest(NullProto proto) throws RemoteException;
-  }
-
-  public static class DummyServer implements DummyServerInterface {
-    @Override
-    public MulResponse mul(MulRequest1 req1) {
-      int x1_1 = req1.getX1();
-      int x1_2 = req1.getX2();
-
-      int result1 = x1_1 * x1_2;
-
-      MulResponse rst = MulResponse.newBuilder().setResult1(result1)
-          .setResult2(400).build();
-      return rst;
-    }
-
-    public void nullParameterTest(NullProto proto) {
-    }
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    // 2. Write Server Part source code
-    server = NettyRpc.getProtoParamRpcServer(new DummyServer(),
-        DummyServerInterface.class, new InetSocketAddress(0));
-    server.start();
-
-    InetSocketAddress addr = server.getBindAddress();
-    Thread.sleep(100);
-
-    // 3. Write client Part source code
-    // 3.1 Make Proxy to make connection to server
-    proxy = (DummyClientInterface) NettyRpc.getProtoParamBlockingRpcProxy(
-        DummyClientInterface.class, addr);
-  }
-
-  @After
-  public void tearDown() throws IOException {
-    server.shutdown();
-  }
-
-  @Test
-  public void testRpcProtoType() throws Exception {
-    // 3.2 Fill request data
-    MulRequest1 req1 = MulRequest1.newBuilder().setX1(10).setX2(20).build();
-
-    // 3.3 call procedure
-    MulResponse re = proxy.mul(req1);
-    assertEquals(200, re.getResult1());
-    assertEquals(400, re.getResult2());
-  }
-
-  @Test
-  public void testNullParameter() throws Exception {
-    NullProto np = NullProto.newBuilder().build();
-
-    // 3.3 call procedure
-    proxy.nullParameterTest(np);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestProtoBlockingRpc.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestProtoBlockingRpc.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestProtoBlockingRpc.java
index 711904b..eeecc3d 100644
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestProtoBlockingRpc.java
+++ b/tajo-rpc/src/test/java/org/apache/tajo/rpc/TestProtoBlockingRpc.java
@@ -30,6 +30,8 @@ import org.apache.tajo.rpc.test.TestProtos.SumResponse;
 import org.apache.tajo.rpc.test.impl.DummyProtocolBlockingImpl;
 
 import java.net.InetSocketAddress;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.*;
 
@@ -55,8 +57,12 @@ public class TestProtoBlockingRpc {
 
   @AfterClass
   public static void tearDown() throws Exception {
-    client.close();
-    server.shutdown();
+    if(client != null) {
+      client.close();
+    }
+    if(server != null) {
+      server.shutdown();
+    }
   }
 
   @Test
@@ -81,6 +87,61 @@ public class TestProtoBlockingRpc {
     assertTrue(service.getNullCalled);
   }
 
+  @Test
+  public void testShutdown() throws Exception {
+    final StringBuilder error = new StringBuilder();
+    Thread callThread = new Thread() {
+      public void run() {
+        try {
+          EchoMessage message = EchoMessage.newBuilder()
+              .setMessage(MESSAGE)
+              .build();
+          stub.deley(null, message);
+//          client.close();
+//          client = null;
+        } catch (Exception e) {
+          e.printStackTrace();
+          error.append(e.getMessage());
+        }
+        synchronized(error) {
+          error.notifyAll();
+        }
+      }
+    };
+
+    callThread.start();
+
+    final CountDownLatch latch = new CountDownLatch(1);
+    Thread shutdownThread = new Thread() {
+      public void run() {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException e) {
+        }
+        try {
+          server.shutdown();
+          server = null;
+          latch.countDown();
+        } catch (Throwable e) {
+          e.printStackTrace();
+        }
+      }
+    };
+    shutdownThread.start();
+
+    latch.await(10 * 1000, TimeUnit.MILLISECONDS);
+
+    assertTrue(latch.getCount() == 0);
+
+    synchronized(error) {
+      error.wait(10 * 1000);
+    }
+
+    if(!error.toString().isEmpty()) {
+      fail(error.toString());
+    }
+  }
+
   //@Test
   public void testGetError() throws Exception {
     EchoMessage echoMessage2 = EchoMessage.newBuilder()

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java b/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
index 6155727..97d410b 100644
--- a/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
+++ b/tajo-rpc/src/test/java/org/apache/tajo/rpc/test/impl/DummyProtocolBlockingImpl.java
@@ -70,7 +70,7 @@ public class DummyProtocolBlockingImpl implements BlockingInterface {
     try {
       Thread.sleep(3000);
     } catch (InterruptedException e) {
-      e.printStackTrace();
+      //throw new ServiceException(e.getMessage(), e);
     }
 
     return request;


[7/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..b935eb7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
@@ -0,0 +1,429 @@
+/**
+ * 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.*;
+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.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.*;
+
+public abstract class ContainerProxy {
+  private 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 {
+    PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
+  }
+
+  private final YarnRPC yarnRPC;
+  private Configuration conf;
+  private QueryMaster.QueryContext context;
+
+  private 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 abstract void containerStarted();
+  protected abstract String getId();
+  protected abstract String getRunnerClass();
+  protected abstract Vector<CharSequence> getTaskParams();
+
+  public ContainerProxy(QueryMaster.QueryContext context, Configuration conf, YarnRPC yarnRPC, Container container) {
+    this.context = context;
+    this.conf = conf;
+    this.yarnRPC = yarnRPC;
+    this.state = ContainerState.PREP;
+    this.container = container;
+    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) {
+    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");
+
+    Path queryConfPath = new Path(fs.getHomeDirectory(), QueryConf.FILENAME);
+    try {
+      writeConf(conf, queryConfPath);
+
+      LocalResource queryConfSrc = createApplicationResource(fsCtx, queryConfPath, LocalResourceType.FILE);
+      localResources.put(QueryConf.FILENAME,  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;
+  }
+
+  public int getTaskPort() {
+    return this.port;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 2f4f12f..3d9a364 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
@@ -51,6 +51,7 @@ 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.master.TajoMaster.MasterContext;
+import org.apache.tajo.master.querymaster.QueryMasterManager;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.storage.StorageUtil;
 import org.apache.tajo.util.TajoIdUtils;
@@ -93,8 +94,7 @@ public class GlobalEngine extends AbstractService {
       planner = new LogicalPlanner(context.getCatalog());
       optimizer = new LogicalOptimizer();
 
-      globalPlanner = new GlobalPlanner(context.getConf(), context.getCatalog(),
-          sm, context.getEventHandler());
+      globalPlanner = new GlobalPlanner(context.getConf(), sm, context.getEventHandler());
 
       globalOptimizer = new GlobalOptimizer();
     } catch (Throwable t) {
@@ -105,7 +105,9 @@ public class GlobalEngine extends AbstractService {
 
   public void stop() {
     super.stop();
-    yarnClient.stop();
+    if (yarnClient != null) {
+      yarnClient.stop();
+    }
   }
 
   public QueryId executeQuery(String tql)
@@ -123,21 +125,26 @@ public class GlobalEngine extends AbstractService {
       updateQuery(plan.getSubNode());
       return TajoIdUtils.NullQueryId;
     } else {
-      ApplicationAttemptId appAttemptId = submitQuery();
-      QueryId queryId = TajoIdUtils.createQueryId(appAttemptId);
-      MasterPlan masterPlan = createGlobalPlan(queryId, plan);
+      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.getSubNode().getType() == ExprType.CREATE_TABLE) {
         CreateTableNode createTableNode = (CreateTableNode) plan.getSubNode();
         queryConf.setOutputTable(createTableNode.getTableName());
       }
-
-      QueryMaster query = new QueryMaster(context, appAttemptId,
-          context.getClock(), querySubmittionTime, masterPlan);
-      startQuery(queryId, queryConf, query);
+      QueryMasterManager queryMasterManager = new QueryMasterManager(context, yarnClient, queryId, tql, plan, appId,
+              context.getClock(), querySubmittionTime);
+      queryMasterManager.init(queryConf);
+      queryMasterManager.start();
+      context.addQuery(queryId, queryMasterManager);
 
       return queryId;
     }
@@ -145,9 +152,46 @@ public class GlobalEngine extends AbstractService {
 
   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);
+
+    // 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;
+  }
+
+  private ApplicationAttemptId submitQueryOld() throws YarnRemoteException {
+    GetNewApplicationResponse newApp = getNewApplication();
     // Get a new application id
     ApplicationId appId = newApp.getApplicationId();
-    System.out.println("Get AppId: " + appId);
+    LOG.info("Get AppId: " + appId);
     LOG.info("Setting up application submission context for ASM");
     ApplicationSubmissionContext appContext = Records
         .newRecord(ApplicationSubmissionContext.class);
@@ -209,7 +253,6 @@ public class GlobalEngine extends AbstractService {
         CreateTableNode createTable = (CreateTableNode) root;
         createTable(createTable);
         return true;
-
       case DROP_TABLE:
         DropTableNode stmt = (DropTableNode) root;
         dropTable(stmt.getTableName());
@@ -227,7 +270,7 @@ public class GlobalEngine extends AbstractService {
     try {
       optimizedPlan = optimizer.optimize(plan);
     } catch (PlanningException e) {
-      e.printStackTrace();
+      LOG.error(e.getMessage(), e);
     }
     LOG.info("LogicalPlan:\n" + plan.getRootBlock().getRoot());
 
@@ -240,12 +283,12 @@ public class GlobalEngine extends AbstractService {
     return globalOptimizer.optimize(globalPlan);
   }
 
-  private void startQuery(final QueryId queryId, final QueryConf queryConf,
-                          final QueryMaster query) {
-    context.getAllQueries().put(queryId, query);
-    query.init(queryConf);
-    query.start();
-  }
+//  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;
@@ -328,7 +371,7 @@ public class GlobalEngine extends AbstractService {
 
   private void connectYarnClient() {
     this.yarnClient = new YarnClientImpl();
-    this.yarnClient.init(getConfig());
+    this.yarnClient.init(context.getConf());
     this.yarnClient.start();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 9907377..9522086 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
@@ -47,7 +47,7 @@ public class GlobalPlanner {
   private StorageManager sm;
   private QueryId queryId;
 
-  public GlobalPlanner(final TajoConf conf, final CatalogService catalog,
+  public GlobalPlanner(final TajoConf conf,
                        final StorageManager sm,
                        final EventHandler eventHandler)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Query.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Query.java
deleted file mode 100644
index 6e7092c..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Query.java
+++ /dev/null
@@ -1,409 +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 com.google.common.collect.Maps;
-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.Path;
-import org.apache.hadoop.yarn.Clock;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-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;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.master.QueryMaster.QueryContext;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.storage.StorageManager;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-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 final EventHandler eventHandler;
-  private final MasterPlan plan;
-  private final StorageManager sm;
-  private QueryContext context;
-  private ExecutionBlockCursor cursor;
-
-  // Query Status
-  private final QueryId id;
-  private long appSubmitTime;
-  private long startTime;
-  private long initializationTime;
-  private long finishTime;
-  private TableDesc resultDesc;
-  private int completedSubQueryCount = 0;
-  private final List<String> diagnostics = new ArrayList<String>();
-
-  // Internal Variables
-  private final Lock readLock;
-  private final Lock writeLock;
-  private int priority = 100;
-
-  // State Machine
-  private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
-
-  private static final StateMachineFactory
-      <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
-      new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
-          (QueryState.QUERY_NEW)
-
-      .addTransition(QueryState.QUERY_NEW,
-          EnumSet.of(QueryState.QUERY_INIT, QueryState.QUERY_FAILED),
-          QueryEventType.INIT, new InitTransition())
-
-      .addTransition(QueryState.QUERY_INIT, QueryState.QUERY_RUNNING,
-          QueryEventType.START, new StartTransition())
-
-      .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
-          QueryEventType.INIT_COMPLETED, new InitCompleteTransition())
-      .addTransition(QueryState.QUERY_RUNNING,
-          EnumSet.of(QueryState.QUERY_RUNNING, QueryState.QUERY_SUCCEEDED,
-              QueryState.QUERY_FAILED),
-          QueryEventType.SUBQUERY_COMPLETED,
-          new SubQueryCompletedTransition())
-      .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
-          QueryEventType.INTERNAL_ERROR, new InternalErrorTransition())
-       .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
-          QueryEventType.INTERNAL_ERROR)
-
-      .installTopology();
-
-  public Query(final QueryContext context, final QueryId id, Clock clock,
-               final long appSubmitTime,
-               final String queryStr,
-               final EventHandler eventHandler,
-               final MasterPlan plan, final StorageManager sm) {
-    this.context = context;
-    this.conf = context.getConf();
-    this.id = id;
-    this.clock = clock;
-    this.appSubmitTime = appSubmitTime;
-    this.queryStr = queryStr;
-    subqueries = Maps.newHashMap();
-    this.eventHandler = eventHandler;
-    this.plan = plan;
-    this.sm = sm;
-    cursor = new ExecutionBlockCursor(plan);
-
-    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
-    this.readLock = readWriteLock.readLock();
-    this.writeLock = readWriteLock.writeLock();
-
-    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()];
-      boolean finished = true;
-      for (SubQuery subquery: subqueries.values()) {
-        if (subquery.getState() != SubQueryState.NEW) {
-          subProgresses[idx] = subquery.getProgress();
-          if (finished && subquery.getState() != SubQueryState.SUCCEEDED) {
-            finished = false;
-          }
-        } else {
-          subProgresses[idx] = 0.0f;
-        }
-        idx++;
-      }
-
-      if (finished) {
-        return 1.0f;
-      }
-
-      float totalProgress = 0;
-      float proportion = 1.0f / (float)subqueries.size();
-
-      for (int i = 0; i < subProgresses.length; i++) {
-        totalProgress += subProgresses[i] * proportion;
-      }
-
-      return totalProgress;
-    }
-  }
-
-  public long getAppSubmitTime() {
-    return this.appSubmitTime;
-  }
-
-  public long getStartTime() {
-    return startTime;
-  }
-
-  public void setStartTime() {
-    startTime = clock.getTime();
-  }
-
-  public long getInitializationTime() {
-    return initializationTime;
-  }
-
-  public void setInitializationTime() {
-    initializationTime = clock.getTime();
-  }
-
-
-  public long getFinishTime() {
-    return finishTime;
-  }
-
-  public void setFinishTime() {
-    finishTime = clock.getTime();
-  }
-
-  public List<String> getDiagnostics() {
-    readLock.lock();
-    try {
-      return diagnostics;
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  protected void addDiagnostic(String diag) {
-    diagnostics.add(diag);
-  }
-
-  public TableDesc getResultDesc() {
-    return resultDesc;
-  }
-
-  public void setResultDesc(TableDesc desc) {
-    resultDesc = desc;
-  }
-
-  public MasterPlan getPlan() {
-    return plan;
-  }
-
-  public StateMachine<QueryState, QueryEventType, QueryEvent> getStateMachine() {
-    return stateMachine;
-  }
-  
-  public void addSubQuery(SubQuery subquery) {
-    subqueries.put(subquery.getId(), subquery);
-  }
-  
-  public QueryId getId() {
-    return this.id;
-  }
-  
-  public SubQuery getSubQuery(SubQueryId id) {
-    return this.subqueries.get(id);
-  }
-
-  public QueryState getState() {
-    readLock.lock();
-    try {
-      return stateMachine.getCurrentState();
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  public ExecutionBlockCursor getExecutionBlockCursor() {
-    return cursor;
-  }
-
-  static class InitTransition
-      implements MultipleArcTransition<Query, QueryEvent, QueryState> {
-
-    @Override
-    public QueryState transition(Query query, QueryEvent queryEvent) {
-      query.setStartTime();
-      return QueryState.QUERY_INIT;
-    }
-  }
-
-  public static class StartTransition
-      implements SingleArcTransition<Query, QueryEvent> {
-
-    @Override
-    public void transition(Query query, QueryEvent queryEvent) {
-      SubQuery subQuery = new SubQuery(query.context, query.getExecutionBlockCursor().nextBlock(),
-          query.sm);
-      subQuery.setPriority(query.priority--);
-      query.addSubQuery(subQuery);
-      LOG.info("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
-      subQuery.handle(new SubQueryEvent(subQuery.getId(),
-          SubQueryEventType.SQ_INIT));
-    }
-  }
-
-  public static class SubQueryCompletedTransition implements
-      MultipleArcTransition<Query, QueryEvent, QueryState> {
-
-    @Override
-    public QueryState transition(Query query, QueryEvent event) {
-      // increase the count for completed subqueries
-      query.completedSubQueryCount++;
-      SubQueryCompletedEvent castEvent = (SubQueryCompletedEvent) event;
-      ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
-
-      // if the subquery is succeeded
-      if (castEvent.getFinalState() == SubQueryState.SUCCEEDED) {
-        if (cursor.hasNext()) {
-          SubQuery nextSubQuery = new SubQuery(query.context, cursor.nextBlock(), query.sm);
-          nextSubQuery.setPriority(query.priority--);
-          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());
-          return query.checkQueryForCompleted();
-
-        } else { // Finish a query
-          if (query.checkQueryForCompleted() == QueryState.QUERY_SUCCEEDED) {
-            SubQuery subQuery = query.getSubQuery(castEvent.getSubQueryId());
-            TableDesc desc = new TableDescImpl(query.conf.getOutputTable(),
-                subQuery.getTableMeta(), query.context.getOutputPath());
-            query.setResultDesc(desc);
-            try {
-              query.writeStat(query.context.getOutputPath(), subQuery);
-            } catch (IOException e) {
-              e.printStackTrace();
-            }
-            query.eventHandler.handle(new QueryFinishEvent(query.getId()));
-
-            if (query.context.isCreateTableQuery()) {
-              query.context.getCatalog().addTable(desc);
-            }
-          }
-
-          return query.finished(QueryState.QUERY_SUCCEEDED);
-        }
-      } else {
-        // if at least one subquery is failed, the query is also failed.
-        return QueryState.QUERY_FAILED;
-      }
-    }
-  }
-
-  private static class DiagnosticsUpdateTransition implements
-      SingleArcTransition<Query, QueryEvent> {
-    @Override
-    public void transition(Query query, QueryEvent event) {
-      query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event)
-          .getDiagnosticUpdate());
-    }
-  }
-
-  private static class InitCompleteTransition implements
-      SingleArcTransition<Query, QueryEvent> {
-    @Override
-    public void transition(Query query, QueryEvent event) {
-      if (query.initializationTime == 0) {
-        query.setInitializationTime();
-      }
-    }
-  }
-
-  private static class InternalErrorTransition
-      implements SingleArcTransition<Query, QueryEvent> {
-
-    @Override
-    public void transition(Query query, QueryEvent event) {
-      query.finished(QueryState.QUERY_ERROR);
-    }
-  }
-
-  public QueryState finished(QueryState finalState) {
-    setFinishTime();
-    return finalState;
-  }
-
-  /**
-   * Check if all subqueries of the query are completed
-   * @return QueryState.QUERY_SUCCEEDED if all subqueries are completed.
-   */
-  QueryState checkQueryForCompleted() {
-    if (completedSubQueryCount == subqueries.size()) {
-      return QueryState.QUERY_SUCCEEDED;
-    }
-    return getState();
-  }
-
-
-  @Override
-  public void handle(QueryEvent event) {
-    LOG.info("Processing " + event.getQueryId() + " of type " + event.getType());
-    try {
-      writeLock.lock();
-      QueryState oldState = getState();
-      try {
-        getStateMachine().doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
-        LOG.error("Can't handle this event at current state", e);
-        eventHandler.handle(new QueryEvent(this.id,
-            QueryEventType.INTERNAL_ERROR));
-      }
-
-      //notify the eventhandler of state change
-      if (oldState != getState()) {
-        LOG.info(id + " Query Transitioned from " + oldState + " to "
-            + getState());
-      }
-    }
-
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  private void writeStat(Path outputPath, SubQuery subQuery)
-      throws IOException {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    sm.writeTableMeta(outputPath, subQuery.getTableMeta());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryMaster.java
deleted file mode 100644
index f4dc455..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryMaster.java
+++ /dev/null
@@ -1,465 +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.lang.exception.ExceptionUtils;
-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.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-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.service.CompositeService;
-import org.apache.hadoop.yarn.service.Service;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.CatalogService;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.TaskRunnerLauncherImpl.ContainerProxy;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.rm.RMContainerAllocator;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.TajoIdUtils;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-public class QueryMaster extends CompositeService implements EventHandler {
-  private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName());
-
-  // Master Context
-  private final MasterContext masterContext;
-
-  // AppMaster Common
-  private final Clock clock;
-  private final long appSubmitTime;
-  private String appName;
-  private final ApplicationAttemptId appAttemptID;
-
-  // For Query
-  private final QueryId queryId;
-  private QueryContext queryContext;
-  private Query query;
-  private MasterPlan masterPlan;
-
-  private AsyncDispatcher dispatcher;
-  private YarnRPC rpc;
-  private RMContainerAllocator rmAllocator;
-  private TaskRunnerListener taskRunnerListener;
-  private TaskRunnerLauncher taskRunnerLauncher;
-
-  // Services of Tajo
-  private CatalogService catalog;
-
-  private boolean isCreateTableStmt;
-  private StorageManager storageManager;
-  private FileSystem defaultFS;
-  private Path outputPath;
-
-  public QueryMaster(final MasterContext masterContext,
-                     final ApplicationAttemptId appAttemptID,
-                     final Clock clock, long appSubmitTime,
-                     MasterPlan masterPlan) {
-    super(QueryMaster.class.getName());
-    this.masterContext = masterContext;
-
-    this.appAttemptID = appAttemptID;
-    this.clock = clock;
-    this.appSubmitTime = appSubmitTime;
-
-    this.queryId = TajoIdUtils.createQueryId(appAttemptID);
-    this.masterPlan = masterPlan;
-    LOG.info("Created Query Master for " + appAttemptID);
-  }
-
-  public void init(Configuration _conf) {
-    QueryConf conf = new QueryConf(_conf);
-
-    try {
-      queryContext = new QueryContext(conf);
-
-      dispatcher = masterContext.getDispatcher();
-      // TODO - This comment should be eliminated when QueryMaster is separated.
-      dispatcher = new AsyncDispatcher();
-      addIfService(dispatcher);
-
-      // TODO - This comment should be improved when QueryMaster is separated.
-      rpc = masterContext.getYarnRPC();
-
-      catalog = masterContext.getCatalog();
-      storageManager = masterContext.getStorageManager();
-
-      taskRunnerListener = new TaskRunnerListener(queryContext);
-      addIfService(taskRunnerListener);
-
-      rmAllocator = new RMContainerAllocator(queryContext);
-      addIfService(rmAllocator);
-      dispatcher.register(ContainerAllocatorEventType.class, rmAllocator);
-
-      query = new Query(queryContext, queryId, clock, appSubmitTime,
-          "", dispatcher.getEventHandler(), masterPlan, storageManager);
-      initStagingDir();
-
-      // QueryEventDispatcher is already registered in TajoMaster
-      dispatcher.register(QueryEventType.class, query);
-      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());
-
-      taskRunnerLauncher = new TaskRunnerLauncherImpl(queryContext);
-      addIfService(taskRunnerLauncher);
-      dispatcher.register(TaskRunnerGroupEvent.EventType.class, taskRunnerLauncher);
-
-
-    } catch (Throwable t) {
-      LOG.error(ExceptionUtils.getStackTrace(t));
-      throw new RuntimeException(t);
-    }
-
-    super.init(conf);
-  }
-
-  public void start() {
-    super.start();
-    startQuery();
-  }
-
-  public void stop() {
-    super.stop();
-  }
-
-  protected void addIfService(Object object) {
-    if (object instanceof Service) {
-      addService((Service) object);
-    }
-  }
-
-  public void startQuery() {
-    dispatcher.getEventHandler().handle(new QueryEvent(queryId,
-        QueryEventType.INIT));
-    dispatcher.getEventHandler().handle(new QueryEvent(queryId,
-        QueryEventType.START));
-  }
-
-  @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);
-    }
-  }
-
-  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 QueryContext getContext() {
-    return this.queryContext;
-  }
-
-  public class QueryContext {
-    private QueryConf conf;
-    public Map<ContainerId, ContainerProxy> containers = new ConcurrentHashMap<ContainerId, ContainerProxy>();
-    int minCapability;
-    int maxCapability;
-    int numCluster;
-
-    public QueryContext(QueryConf conf) {
-      this.conf = conf;
-    }
-
-    public QueryConf getConf() {
-      return conf;
-    }
-
-    public AsyncDispatcher getDispatcher() {
-      return dispatcher;
-    }
-
-    public Clock getClock() {
-      return clock;
-    }
-
-    public Query getQuery() {
-      return query;
-    }
-
-    public SubQuery getSubQuery(SubQueryId subQueryId) {
-      return query.getSubQuery(subQueryId);
-    }
-
-    public QueryId getQueryId() {
-      return queryId;
-    }
-
-    public ApplicationId getApplicationId() {
-      return appAttemptID.getApplicationId();
-    }
-
-    public ApplicationAttemptId getApplicationAttemptId() {
-      return appAttemptID;
-    }
-
-    public EventHandler getEventHandler() {
-      return dispatcher.getEventHandler();
-    }
-
-    public YarnRPC getYarnRPC() {
-      return rpc;
-    }
-
-    public InetSocketAddress getRpcAddress() {
-      return masterContext.getClientService().getBindAddress();
-    }
-
-    public InetSocketAddress getTaskListener() {
-      return taskRunnerListener.getBindAddress();
-    }
-
-    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 int getNumClusterNode() {
-      return numCluster;
-    }
-
-    public void setNumClusterNodes(int num) {
-      numCluster = num;
-    }
-
-    public CatalogService getCatalog() {
-      return catalog;
-    }
-
-    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() {
-      return query.getProgress();
-    }
-
-    public long getStartTime() {
-      return query.getStartTime();
-    }
-
-    public long getFinishTime() {
-      return query.getFinishTime();
-    }
-
-    public StorageManager getStorageManager() {
-      return storageManager;
-    }
-  }
-
-  private class QueryFinishEventHandler implements EventHandler<QueryFinishEvent> {
-    @Override
-    public void handle(QueryFinishEvent event) {
-      LOG.info("Query end notification started for QueryId : " + query.getId());
-
-      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);
-      }
-
-      //Bring the process down by force.
-      //Not needed after HADOOP-7140
-      LOG.info("Exiting QueryMaster..GoodBye!");
-      // TODO - to be enabled if query master is separated.
-      //System.exit(0);
-    }
-  }
-
-  // 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();
-
-        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
-      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());
-
-      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));
-      }
-    }
-
-    conf.setOutputPath(stagingDir);
-    outputPath = stagingDir;
-    LOG.info("Initialized Query Staging Dir: " + outputPath);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnit.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnit.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnit.java
deleted file mode 100644
index 8eb26bc..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnit.java
+++ /dev/null
@@ -1,499 +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 com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-import org.apache.tajo.QueryIdFactory;
-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.engine.MasterWorkerProtos.Partition;
-import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.storage.Fragment;
-
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class QueryUnit implements EventHandler<TaskEvent> {
-  /** Class Logger */
-  private static final Log LOG = LogFactory.getLog(QueryUnit.class);
-
-	private QueryUnitId taskId;
-  private EventHandler eventHandler;
-	private StoreTableNode store = null;
-	private LogicalNode plan = null;
-	private List<ScanNode> scan;
-	
-	private Map<String, Fragment> fragMap;
-	private Map<String, Set<URI>> fetchMap;
-	
-  private List<Partition> partitions;
-	private TableStat stats;
-  private String [] dataLocations;
-  private final boolean isLeafTask;
-  private List<IntermediateEntry> intermediateData;
-
-  private Map<QueryUnitAttemptId, QueryUnitAttempt> attempts;
-  private final int maxAttempts = 3;
-  private Integer lastAttemptId;
-
-  private QueryUnitAttemptId successfulAttempt;
-  private String succeededHost;
-  private int succeededPullServerPort;
-
-  private int failedAttempts;
-  private int finishedAttempts; // finish are total of success, failed and killed
-
-  private static final StateMachineFactory
-      <QueryUnit, TaskState, TaskEventType, TaskEvent> stateMachineFactory =
-      new StateMachineFactory
-          <QueryUnit, TaskState, TaskEventType, TaskEvent>(TaskState.NEW)
-
-      .addTransition(TaskState.NEW, TaskState.SCHEDULED,
-          TaskEventType.T_SCHEDULE, new InitialScheduleTransition())
-
-       .addTransition(TaskState.SCHEDULED, TaskState.RUNNING,
-           TaskEventType.T_ATTEMPT_LAUNCHED)
-
-        .addTransition(TaskState.RUNNING, TaskState.RUNNING,
-           TaskEventType.T_ATTEMPT_LAUNCHED)
-
-       .addTransition(TaskState.RUNNING, TaskState.SUCCEEDED,
-           TaskEventType.T_ATTEMPT_SUCCEEDED, new AttemptSucceededTransition())
-
-       .addTransition(TaskState.RUNNING,
-            EnumSet.of(TaskState.RUNNING, TaskState.FAILED),
-            TaskEventType.T_ATTEMPT_FAILED, new AttemptFailedTransition())
-
-
-
-      .installTopology();
-  private final StateMachine<TaskState, TaskEventType, TaskEvent> stateMachine;
-
-
-  private final Lock readLock;
-  private final Lock writeLock;
-
-	public QueryUnit(QueryUnitId id, boolean isLeafTask, EventHandler eventHandler) {
-		this.taskId = id;
-    this.eventHandler = eventHandler;
-    this.isLeafTask = isLeafTask;
-		scan = new ArrayList<ScanNode>();
-    fetchMap = Maps.newHashMap();
-    fragMap = Maps.newHashMap();
-    partitions = new ArrayList<Partition>();
-    attempts = Collections.emptyMap();
-    lastAttemptId = -1;
-    failedAttempts = 0;
-
-    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
-    this.readLock = readWriteLock.readLock();
-    this.writeLock = readWriteLock.writeLock();
-
-    stateMachine = stateMachineFactory.make(this);
-	}
-
-  public boolean isLeafTask() {
-    return this.isLeafTask;
-  }
-
-  public void setDataLocations(String [] dataLocations) {
-    this.dataLocations = dataLocations;
-  }
-
-  public String [] getDataLocations() {
-    return this.dataLocations;
-  }
-
-  public TaskState getState() {
-    readLock.lock();
-    try {
-      return stateMachine.getCurrentState();
-    } finally {
-      readLock.unlock();
-    }
-  }
-	
-	public void setLogicalPlan(LogicalNode plan) {
-    Preconditions.checkArgument(plan.getType() == ExprType.STORE ||
-        plan.getType() == ExprType.CREATE_INDEX);
-    
-	  this.plan = plan;
-    store = (StoreTableNode) plan;
-
-	  LogicalNode node = plan;
-	  ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
-	  s.add(node);
-	  while (!s.isEmpty()) {
-	    node = s.remove(s.size()-1);
-	    if (node instanceof UnaryNode) {
-	      UnaryNode unary = (UnaryNode) node;
-	      s.add(s.size(), unary.getSubNode());
-	    } else if (node instanceof BinaryNode) {
-	      BinaryNode binary = (BinaryNode) node;
-	      s.add(s.size(), binary.getOuterNode());
-	      s.add(s.size(), binary.getInnerNode());
-	    } else if (node instanceof ScanNode) {
-	      scan.add((ScanNode)node);
-	    }
-	  }
-	}
-
-  @Deprecated
-  public void setFragment(String tableId, Fragment fragment) {
-    this.fragMap.put(tableId, fragment);
-    if (fragment.hasDataLocations()) {
-      setDataLocations(fragment.getDataLocations());
-    }
-  }
-
-  public void setFragment2(Fragment fragment) {
-    this.fragMap.put(fragment.getId(), fragment);
-    if (fragment.hasDataLocations()) {
-      setDataLocations(fragment.getDataLocations());
-    }
-  }
-	
-	public void addFetch(String tableId, String uri) throws URISyntaxException {
-	  this.addFetch(tableId, new URI(uri));
-	}
-	
-	public void addFetch(String tableId, URI uri) {
-	  Set<URI> uris;
-	  if (fetchMap.containsKey(tableId)) {
-	    uris = fetchMap.get(tableId);
-	  } else {
-	    uris = Sets.newHashSet();
-	  }
-	  uris.add(uri);
-    fetchMap.put(tableId, uris);
-	}
-	
-	public void addFetches(String tableId, Collection<URI> urilist) {
-	  Set<URI> uris;
-    if (fetchMap.containsKey(tableId)) {
-      uris = fetchMap.get(tableId);
-    } else {
-      uris = Sets.newHashSet();
-    }
-    uris.addAll(urilist);
-    fetchMap.put(tableId, uris);
-	}
-	
-	public void setFetches(Map<String, Set<URI>> fetches) {
-	  this.fetchMap.clear();
-	  this.fetchMap.putAll(fetches);
-	}
-	
-  public Fragment getFragment(String tableId) {
-    return this.fragMap.get(tableId);
-  }
-
-  public Collection<Fragment> getAllFragments() {
-    return fragMap.values();
-  }
-	
-	public LogicalNode getLogicalPlan() {
-	  return this.plan;
-	}
-	
-	public QueryUnitId getId() {
-		return taskId;
-	}
-	
-	public Collection<URI> getFetchHosts(String tableId) {
-	  return fetchMap.get(tableId);
-	}
-	
-	public Collection<Set<URI>> getFetches() {
-	  return fetchMap.values();
-	}
-	
-	public Collection<URI> getFetch(ScanNode scan) {
-	  return this.fetchMap.get(scan.getTableId());
-	}
-
-	public String getOutputName() {
-		return this.store.getTableName();
-	}
-	
-	public Schema getOutputSchema() {
-	  return this.store.getOutSchema();
-	}
-	
-	public StoreTableNode getStoreTableNode() {
-	  return this.store;
-	}
-	
-	public ScanNode[] getScanNodes() {
-	  return this.scan.toArray(new ScanNode[scan.size()]);
-	}
-	
-	@Override
-	public String toString() {
-		String str = new String(plan.getType() + " \n");
-		for (Entry<String, Fragment> e : fragMap.entrySet()) {
-		  str += e.getKey() + " : ";
-      str += e.getValue() + " ";
-		}
-		for (Entry<String, Set<URI>> e : fetchMap.entrySet()) {
-      str += e.getKey() + " : ";
-      for (URI t : e.getValue()) {
-        str += t + " ";
-      }
-    }
-		
-		return str;
-	}
-	
-	public void setStats(TableStat stats) {
-	  this.stats = stats;
-	}
-	
-	public void setPartitions(List<Partition> partitions) {
-	  this.partitions = Collections.unmodifiableList(partitions);
-	}
-	
-	public TableStat getStats() {
-	  return this.stats;
-	}
-	
-	public List<Partition> getPartitions() {
-	  return this.partitions;
-	}
-	
-	public int getPartitionNum() {
-	  return this.partitions.size();
-	}
-
-  public QueryUnitAttempt newAttempt() {
-    QueryUnitAttempt attempt = new QueryUnitAttempt(
-        QueryIdFactory.newQueryUnitAttemptId(this.getId(),
-            ++lastAttemptId), this, eventHandler);
-    return attempt;
-  }
-
-  public QueryUnitAttempt getAttempt(QueryUnitAttemptId attemptId) {
-    return attempts.get(attemptId);
-  }
-
-  public QueryUnitAttempt getAttempt(int attempt) {
-    return this.attempts.get(new QueryUnitAttemptId(this.getId(), attempt));
-  }
-
-  public QueryUnitAttempt getLastAttempt() {
-    return this.attempts.get(this.lastAttemptId);
-  }
-
-  protected QueryUnitAttempt getSuccessfulAttempt() {
-    readLock.lock();
-    try {
-      if (null == successfulAttempt) {
-        return null;
-      }
-      return attempts.get(successfulAttempt);
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  public int getRetryCount () {
-    return this.lastAttemptId;
-  }
-
-  private static class InitialScheduleTransition implements
-    SingleArcTransition<QueryUnit, TaskEvent> {
-
-    @Override
-    public void transition(QueryUnit task, TaskEvent taskEvent) {
-      task.addAndScheduleAttempt();
-    }
-  }
-
-  // This is always called in the Write Lock
-  private void addAndScheduleAttempt() {
-    // Create new task attempt
-    QueryUnitAttempt attempt = newAttempt();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Created attempt " + attempt.getId());
-    }
-    switch (attempts.size()) {
-      case 0:
-        attempts = Collections.singletonMap(attempt.getId(), attempt);
-        break;
-
-      case 1:
-        Map<QueryUnitAttemptId, QueryUnitAttempt> newAttempts
-            = new LinkedHashMap<QueryUnitAttemptId, QueryUnitAttempt>(3);
-        newAttempts.putAll(attempts);
-        attempts = newAttempts;
-        attempts.put(attempt.getId(), attempt);
-        break;
-
-      default:
-        attempts.put(attempt.getId(), attempt);
-        break;
-    }
-
-    if (failedAttempts > 0) {
-      eventHandler.handle(new TaskAttemptEvent(attempt.getId(),
-          TaskAttemptEventType.TA_RESCHEDULE));
-    } else {
-      eventHandler.handle(new TaskAttemptEvent(attempt.getId(),
-          TaskAttemptEventType.TA_SCHEDULE));
-    }
-  }
-
-  private static class AttemptSucceededTransition
-      implements SingleArcTransition<QueryUnit, TaskEvent>{
-
-    @Override
-    public void transition(QueryUnit task,
-                           TaskEvent event) {
-      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
-      QueryUnitAttempt attempt = task.attempts.get(
-          attemptEvent.getTaskAttemptId());
-      task.successfulAttempt = attemptEvent.getTaskAttemptId();
-      task.succeededHost = attempt.getHost();
-      task.succeededPullServerPort = attempt.getPullServerPort();
-      task.eventHandler.handle(new SubQueryTaskEvent(event.getTaskId(),
-          SubQueryEventType.SQ_TASK_COMPLETED));
-    }
-  }
-
-  private static class AttemptFailedTransition implements
-    MultipleArcTransition<QueryUnit, TaskEvent, TaskState> {
-
-    @Override
-    public TaskState transition(QueryUnit task, TaskEvent taskEvent) {
-      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) taskEvent;
-      LOG.info("=============================================================");
-      LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + " <<<");
-      LOG.info("=============================================================");
-      task.failedAttempts++;
-      task.finishedAttempts++;
-
-      if (task.failedAttempts < task.maxAttempts) {
-        if (task.successfulAttempt == null) {
-          task.addAndScheduleAttempt();
-        }
-      } else {
-        task.eventHandler.handle(
-            new SubQueryTaskEvent(task.getId(), SubQueryEventType.SQ_FAILED));
-        return TaskState.FAILED;
-      }
-
-      return task.getState();
-    }
-  }
-
-  @Override
-  public void handle(TaskEvent event) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing " + event.getTaskId() + " of type "
-          + event.getType());
-    }
-
-    try {
-      writeLock.lock();
-      TaskState oldState = getState();
-      try {
-        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(),
-            QueryEventType.INTERNAL_ERROR));
-      }
-
-      //notify the eventhandler of state change
-      if (LOG.isDebugEnabled()) {
-        if (oldState != getState()) {
-          LOG.debug(taskId + " Task Transitioned from " + oldState + " to "
-              + getState());
-        }
-      }
-    }
-
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  public void setIntermediateData(Collection<IntermediateEntry> partitions) {
-    this.intermediateData = new ArrayList<IntermediateEntry>(partitions);
-  }
-
-  public List<IntermediateEntry> getIntermediateData() {
-    return this.intermediateData;
-  }
-
-  public static class IntermediateEntry {
-    int taskId;
-    int attemptId;
-    int partitionId;
-    String pullHost;
-    int port;
-
-    public IntermediateEntry(int taskId, int attemptId, int partitionId,
-                             String pullServerAddr, int pullServerPort) {
-      this.taskId = taskId;
-      this.attemptId = attemptId;
-      this.partitionId = partitionId;
-      this.pullHost = pullServerAddr;
-      this.port = pullServerPort;
-    }
-
-    public int getTaskId() {
-      return this.taskId;
-    }
-
-    public int getAttemptId() {
-      return this.attemptId;
-    }
-
-    public int getPartitionId() {
-      return this.partitionId;
-    }
-
-    public String getPullHost() {
-      return this.pullHost;
-    }
-
-    public int getPullPort() {
-      return port;
-    }
-
-    public String getPullAddress() {
-      return pullHost + ":" + port;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnitAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnitAttempt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnitAttempt.java
deleted file mode 100644
index cf95a9e..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryUnitAttempt.java
+++ /dev/null
@@ -1,344 +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.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-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.engine.MasterWorkerProtos.Partition;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskCompletionReport;
-import org.apache.tajo.master.QueryUnit.IntermediateEntry;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
-
-import java.util.*;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
-
-  private static final Log LOG = LogFactory.getLog(QueryUnitAttempt.class);
-
-  private final static int EXPIRE_TIME = 15000;
-
-  private final QueryUnitAttemptId id;
-  private final QueryUnit queryUnit;
-  final EventHandler eventHandler;
-
-  private String hostName;
-  private int port;
-  private int expire;
-
-  private final Lock readLock;
-  private final Lock writeLock;
-
-  private final List<String> diagnostics = new ArrayList<String>();
-
-  private static final StateMachineFactory
-      <QueryUnitAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
-      stateMachineFactory = new StateMachineFactory
-      <QueryUnitAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
-      (TaskAttemptState.TA_NEW)
-
-      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
-          TaskAttemptEventType.TA_SCHEDULE, new TaskAttemptScheduleTransition())
-      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
-          TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition())
-
-      .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_ASSIGNED,
-          TaskAttemptEventType.TA_ASSIGNED, new LaunchTransition())
-
-      // from assigned
-      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED,
-          TaskAttemptEventType.TA_ASSIGNED, new AlreadyAssignedTransition())
-      .addTransition(TaskAttemptState.TA_ASSIGNED,
-          EnumSet.of(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILLED),
-          TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
-
-      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_SUCCEEDED,
-          TaskAttemptEventType.TA_DONE, new SucceededTransition())
-
-      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_FAILED,
-          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
-
-      // from running
-      .addTransition(TaskAttemptState.TA_RUNNING,
-          EnumSet.of(TaskAttemptState.TA_RUNNING),
-          TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
-
-      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_SUCCEEDED,
-          TaskAttemptEventType.TA_DONE, new SucceededTransition())
-
-      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_FAILED,
-          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
-
-      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
-          TaskAttemptEventType.TA_UPDATE)
-      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
-          TaskAttemptEventType.TA_DONE, new AlreadyDoneTransition())
-      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_FAILED,
-          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
-
-      .installTopology();
-
-  private final StateMachine<TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
-    stateMachine;
-
-
-  public QueryUnitAttempt(final QueryUnitAttemptId id, final QueryUnit queryUnit,
-                          final EventHandler eventHandler) {
-    this.id = id;
-    this.expire = QueryUnitAttempt.EXPIRE_TIME;
-    this.queryUnit = queryUnit;
-    this.eventHandler = eventHandler;
-
-    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
-    this.readLock = readWriteLock.readLock();
-    this.writeLock = readWriteLock.writeLock();
-
-    stateMachine = stateMachineFactory.make(this);
-  }
-
-  public TaskAttemptState getState() {
-    readLock.lock();
-    try {
-      return stateMachine.getCurrentState();
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  public QueryUnitAttemptId getId() {
-    return this.id;
-  }
-
-  public boolean isLeafTask() {
-    return this.queryUnit.isLeafTask();
-  }
-
-  public QueryUnit getQueryUnit() {
-    return this.queryUnit;
-  }
-
-  public String getHost() {
-    return this.hostName;
-  }
-
-  public void setHost(String host) {
-    this.hostName = host;
-  }
-
-  public void setPullServerPort(int port) {
-    this.port = port;
-  }
-
-  public int getPullServerPort() {
-    return port;
-  }
-
-  public synchronized void setExpireTime(int expire) {
-    this.expire = expire;
-  }
-
-  public synchronized void updateExpireTime(int period) {
-    this.setExpireTime(this.expire - period);
-  }
-
-  public synchronized void resetExpireTime() {
-    this.setExpireTime(QueryUnitAttempt.EXPIRE_TIME);
-  }
-
-  public int getLeftTime() {
-    return this.expire;
-  }
-
-  private void fillTaskStatistics(TaskCompletionReport report) {
-    if (report.getPartitionsCount() > 0) {
-      this.getQueryUnit().setPartitions(report.getPartitionsList());
-
-      List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
-      for (Partition p : report.getPartitionsList()) {
-        IntermediateEntry entry = new IntermediateEntry(getId().getQueryUnitId().getId(),
-            getId().getId(), p.getPartitionKey(), getHost(), getPullServerPort());
-        partitions.add(entry);
-      }
-      this.getQueryUnit().setIntermediateData(partitions);
-    }
-    if (report.hasResultStats()) {
-      this.getQueryUnit().setStats(new TableStat(report.getResultStats()));
-    }
-  }
-
-  private static class TaskAttemptScheduleTransition implements
-    SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
-
-    @Override
-    public void transition(QueryUnitAttempt taskAttempt,
-                           TaskAttemptEvent taskAttemptEvent) {
-
-      if (taskAttempt.isLeafTask()
-          && taskAttempt.getQueryUnit().getScanNodes().length == 1) {
-        Set<String> racks = new HashSet<String>();
-        for (String host : taskAttempt.getQueryUnit().getDataLocations()) {
-          racks.add(RackResolver.resolve(host).getNetworkLocation());
-        }
-
-        taskAttempt.eventHandler.handle(new TaskScheduleEvent(
-            taskAttempt.getId(), EventType.T_SCHEDULE, true,
-            taskAttempt.getQueryUnit().getDataLocations(),
-            racks.toArray(new String[racks.size()])
-        ));
-      } else {
-        taskAttempt.eventHandler.handle(new TaskScheduleEvent(
-            taskAttempt.getId(), EventType.T_SCHEDULE,
-            false,
-            null,
-            null
-        ));
-      }
-    }
-  }
-
-  private static class LaunchTransition
-      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
-
-    @Override
-    public void transition(QueryUnitAttempt taskAttempt,
-                           TaskAttemptEvent event) {
-      TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event;
-      taskAttempt.setHost(castEvent.getHostName());
-      taskAttempt.setPullServerPort(castEvent.getPullServerPort());
-      taskAttempt.eventHandler.handle(
-          new TaskTAttemptEvent(taskAttempt.getId(),
-              TaskEventType.T_ATTEMPT_LAUNCHED));
-    }
-  }
-
-  private static class StatusUpdateTransition
-      implements MultipleArcTransition<QueryUnitAttempt, TaskAttemptEvent, TaskAttemptState> {
-
-    @Override
-    public TaskAttemptState transition(QueryUnitAttempt taskAttempt,
-                                       TaskAttemptEvent event) {
-      TaskAttemptStatusUpdateEvent updateEvent =
-          (TaskAttemptStatusUpdateEvent) event;
-
-      switch (updateEvent.getStatus().getState()) {
-        case TA_PENDING:
-        case TA_RUNNING:
-          return TaskAttemptState.TA_RUNNING;
-
-        default:
-          return taskAttempt.getState();
-      }
-    }
-  }
-
-  private void addDiagnosticInfo(String diag) {
-    if (diag != null && !diag.equals("")) {
-      diagnostics.add(diag);
-    }
-  }
-
-  private static class AlreadyAssignedTransition
-      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
-
-    @Override
-    public void transition(QueryUnitAttempt queryUnitAttempt,
-                           TaskAttemptEvent taskAttemptEvent) {
-      LOG.info(">>>>>>>>> Already Assigned: " + queryUnitAttempt.getId());
-    }
-  }
-
-  private static class AlreadyDoneTransition
-      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
-
-    @Override
-    public void transition(QueryUnitAttempt queryUnitAttempt,
-                           TaskAttemptEvent taskAttemptEvent) {
-      LOG.info(">>>>>>>>> Already Done: " + queryUnitAttempt.getId());
-    }
-  }
-
-  private static class SucceededTransition
-      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
-    @Override
-    public void transition(QueryUnitAttempt taskAttempt,
-                           TaskAttemptEvent event) {
-      TaskCompletionReport report = ((TaskCompletionEvent)event).getReport();
-
-      taskAttempt.fillTaskStatistics(report);
-      taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(),
-          TaskEventType.T_ATTEMPT_SUCCEEDED));
-    }
-  }
-
-  private static class FailedTransition
-      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
-    @Override
-    public void transition(QueryUnitAttempt taskAttempt,
-                           TaskAttemptEvent event) {
-      TaskFatalErrorEvent errorEvent = (TaskFatalErrorEvent) event;
-      taskAttempt.eventHandler.handle(
-          new TaskTAttemptEvent(taskAttempt.getId(),
-              TaskEventType.T_ATTEMPT_FAILED));
-      LOG.error("FROM " + taskAttempt.getHost() + " >> "
-          + errorEvent.errorMessage());
-      taskAttempt.addDiagnosticInfo(errorEvent.errorMessage());
-    }
-  }
-
-  @Override
-  public void handle(TaskAttemptEvent event) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing " + event.getTaskAttemptId() + " of type "
-          + event.getType());
-    }
-    try {
-      writeLock.lock();
-      TaskAttemptState oldState = getState();
-      try {
-        stateMachine.doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
-        LOG.error("Can't handle this event at current state of "
-            + event.getTaskAttemptId() + ")", e);
-        eventHandler.handle(new QueryEvent(getId().getQueryId(),
-            QueryEventType.INTERNAL_ERROR));
-      }
-
-      //notify the eventhandler of state change
-      if (LOG.isDebugEnabled()) {
-       if (oldState != getState()) {
-          LOG.debug(id + " TaskAttempt Transitioned from " + oldState + " to "
-              + getState());
-        }
-      }
-    }
-
-    finally {
-      writeLock.unlock();
-    }
-  }
-}


[8/8] git commit: TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
TAJO-91: Launch QueryMaster on NodeManager per query. (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/9d020883
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/9d020883
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/9d020883

Branch: refs/heads/master
Commit: 9d02088397aca145aa77799e24469a983cc60974
Parents: a269372
Author: Hyunsik Choi <hy...@apache.org>
Authored: Wed Aug 14 13:37:18 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Wed Aug 14 13:38:51 2013 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../org/apache/tajo/catalog/CatalogService.java |   4 +-
 .../org/apache/tajo/catalog/FunctionDesc.java   |   2 +-
 .../src/main/proto/CatalogProtos.proto          | 160 ++--
 .../apache/tajo/catalog/TestFunctionDesc.java   |   5 +-
 .../org/apache/tajo/catalog/CatalogServer.java  |   6 +-
 .../src/main/java/org/apache/tajo/QueryId.java  |   4 +-
 .../java/org/apache/tajo/conf/TajoConf.java     |   1 +
 .../main/java/org/apache/tajo/util/Bytes.java   |  14 +-
 .../org/apache/tajo/util/CommonTestingUtil.java |   2 +-
 .../java/org/apache/tajo/util/TajoIdUtils.java  |  40 +-
 tajo-core/tajo-core-backend/pom.xml             |  23 +-
 .../org/apache/tajo/benchmark/BenchmarkSet.java |   9 +
 .../main/java/org/apache/tajo/cli/TajoCli.java  |  33 +-
 .../org/apache/tajo/client/QueryStatus.java     |  15 +-
 .../java/org/apache/tajo/client/TajoClient.java | 162 +++-
 .../apache/tajo/engine/eval/EvalTreeUtil.java   |   6 +-
 .../apache/tajo/engine/planner/FromTable.java   |   4 +
 .../tajo/engine/query/QueryUnitRequestImpl.java |   4 +-
 .../apache/tajo/engine/query/ResultSetImpl.java |  66 +-
 .../ipc/protocolrecords/QueryUnitRequest.java   |   7 +-
 .../org/apache/tajo/master/ClientService.java   | 411 ----------
 .../org/apache/tajo/master/ContainerProxy.java  | 429 ++++++++++
 .../org/apache/tajo/master/GlobalEngine.java    |  85 +-
 .../org/apache/tajo/master/GlobalPlanner.java   |   2 +-
 .../main/java/org/apache/tajo/master/Query.java | 409 ----------
 .../org/apache/tajo/master/QueryMaster.java     | 465 -----------
 .../java/org/apache/tajo/master/QueryUnit.java  | 499 -----------
 .../apache/tajo/master/QueryUnitAttempt.java    | 344 --------
 .../org/apache/tajo/master/Repartitioner.java   | 582 -------------
 .../java/org/apache/tajo/master/SubQuery.java   | 766 -----------------
 .../org/apache/tajo/master/SubQueryState.java   |  28 -
 .../java/org/apache/tajo/master/TajoMaster.java |  64 +-
 .../tajo/master/TajoMasterClientService.java    | 412 ++++++++++
 .../tajo/master/TaskRunnerLauncherImpl.java     | 487 ++---------
 .../apache/tajo/master/TaskRunnerListener.java  | 172 ----
 .../apache/tajo/master/TaskSchedulerImpl.java   |  25 +-
 .../tajo/master/cluster/WorkerListener.java     | 148 ----
 .../master/event/SubQueryCompletedEvent.java    |   2 +-
 .../tajo/master/event/SubQuerySucceeEvent.java  |   2 +-
 .../event/TaskAttemptStatusUpdateEvent.java     |   2 +-
 .../tajo/master/event/TaskCompletionEvent.java  |   2 +-
 .../tajo/master/event/TaskFatalErrorEvent.java  |   2 +-
 .../tajo/master/event/TaskRequestEvent.java     |   2 +-
 .../apache/tajo/master/querymaster/Query.java   | 413 ++++++++++
 .../tajo/master/querymaster/QueryMaster.java    | 817 +++++++++++++++++++
 .../querymaster/QueryMasterClientService.java   | 196 +++++
 .../master/querymaster/QueryMasterManager.java  | 353 ++++++++
 .../querymaster/QueryMasterManagerService.java  | 114 +++
 .../master/querymaster/QueryMasterRunner.java   | 152 ++++
 .../tajo/master/querymaster/QueryUnit.java      | 500 ++++++++++++
 .../master/querymaster/QueryUnitAttempt.java    | 344 ++++++++
 .../tajo/master/querymaster/Repartitioner.java  | 584 +++++++++++++
 .../tajo/master/querymaster/SubQuery.java       | 766 +++++++++++++++++
 .../tajo/master/querymaster/SubQueryState.java  |  28 +
 .../tajo/master/rm/RMContainerAllocator.java    |  31 +-
 .../main/java/org/apache/tajo/worker/Task.java  |   4 +-
 .../java/org/apache/tajo/worker/TaskRunner.java |  61 +-
 .../src/main/proto/CatalogProtos.proto          | 160 ++--
 .../src/main/proto/ClientProtocol.proto         |   7 +-
 .../src/main/proto/ClientProtos.proto           | 136 +++
 .../src/main/proto/MasterWorkerProtocol.proto   |  36 -
 .../src/main/proto/MasterWorkerProtos.proto     | 118 ---
 .../main/proto/QueryMasterClientProtocol.proto  |  36 +
 .../main/proto/QueryMasterManagerProtocol.proto |  50 ++
 .../src/main/proto/QueryMasterProtocol.proto    | 132 +++
 .../main/proto/TajoMasterClientProtocol.proto   |  64 ++
 .../src/main/proto/tajo_protos.proto            |  17 +-
 .../src/main/resources/tajo-default.xml         |  18 +
 .../org/apache/tajo/BackendTestingUtil.java     |   2 +-
 .../apache/tajo/LocalTajoTestingUtility.java    |  12 +
 .../org/apache/tajo/MiniTajoYarnCluster.java    |   5 +
 .../org/apache/tajo/TajoTestingCluster.java     |  63 +-
 .../test/java/org/apache/tajo/TpchTestBase.java |   5 +-
 .../org/apache/tajo/benchmark/TestTPCH.java     |  45 +-
 .../org/apache/tajo/client/TestTajoClient.java  |   4 +-
 .../engine/function/TestBuiltinFunctions.java   |  58 +-
 .../plan/global/TestGlobalQueryPlanner.java     |   2 +-
 .../global/TestGlobalQueryOptimizer.java        |  10 +-
 .../tajo/engine/query/TestGroupByQuery.java     |  43 +-
 .../apache/tajo/engine/query/TestJoinQuery.java | 134 +--
 .../tajo/engine/query/TestNullValues.java       |  34 +-
 .../tajo/engine/query/TestResultSetImpl.java    |   2 +-
 .../tajo/engine/query/TestSelectQuery.java      | 339 ++++----
 .../apache/tajo/engine/query/TestSortQuery.java | 148 ++--
 .../tajo/master/TestExecutionBlockCursor.java   |   2 +-
 .../apache/tajo/master/TestRepartitioner.java   |  10 +-
 .../org/apache/tajo/worker/TaskRunnerTest.java  |   6 +-
 .../src/test/resources/log4j.properties         |   5 +-
 tajo-core/tajo-core-pullserver/pom.xml          |   3 +-
 .../src/main/proto/CatalogProtos.proto          | 160 ++--
 tajo-project/pom.xml                            |   9 +-
 tajo-rpc/pom.xml                                |   3 +-
 .../org/apache/tajo/rpc/NettyServerBase.java    |  52 +-
 .../apache/tajo/rpc/ProtoBlockingRpcServer.java |   4 -
 .../org/apache/tajo/rpc/TestNettyAsyncRpc.java  | 115 ---
 .../apache/tajo/rpc/TestNettyBlockingRpc.java   | 110 ---
 .../apache/tajo/rpc/TestProtoBlockingRpc.java   |  65 +-
 .../test/impl/DummyProtocolBlockingImpl.java    |   2 +-
 99 files changed, 6969 insertions(+), 5530 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 808d9cb..0e84b29 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,9 @@ Release 0.2.0 - unreleased
 
   IMPROVEMENTS
 
+    TAJO-91: Launch QueryMaster on NodeManager per query. 
+    (hyoungjunkim via hyunsik)
+
     TAJO-100: Port the parse error handling to the new parser. (jinho)
 
     TAJO-121: Add LogicalPlanVisitor and Refactor LogicalOptimizer to use the

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
index c4eb535..d0b5f50 100644
--- a/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
+++ b/tajo-catalog/tajo-catalog-client/src/main/java/org/apache/tajo/catalog/CatalogService.java
@@ -28,7 +28,7 @@ public interface CatalogService {
    * Get a table description by name
    * @param name table name
    * @return a table description
-   * @see TableDescImpl
+   * @see org.apache.tajo.catalog.TableDescImpl
    * @throws Throwable
    */
   TableDesc getTableDesc(String name);
@@ -49,7 +49,7 @@ public interface CatalogService {
 
   /**
    * Add a table via table description
-   * @see TableDescImpl
+   * @see org.apache.tajo.catalog.TableDescImpl
    * @throws Throwable
    */
   boolean addTable(TableDesc desc);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
index 8f678b9..73f861a 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java
@@ -67,7 +67,7 @@ public class FunctionDesc implements ProtoObject<FunctionDescProto>, Cloneable,
   /**
    * 
    * @return 함수 인스턴스
-   * @throws InternalException
+   * @throws org.apache.tajo.exception.InternalException
    */
   public Function newInstance() throws InternalException {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index c171c2b..6164553 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -25,9 +25,9 @@ option java_generate_equals_and_hash = true;
 import "DataTypes.proto";
 
 enum StoreType {
-  MEM = 0;
-  CSV = 1;
-  RAW = 2;
+	MEM = 0;
+	CSV = 1;
+	RAW = 2;
   RCFILE = 3;
   ROWFILE = 4;
   HCFILE = 5;
@@ -35,147 +35,147 @@ enum StoreType {
 }
 
 enum OrderType {
-  ORDER_NONE = 0;
-  ASC = 1;
-  DSC = 2;
+    ORDER_NONE = 0;
+    ASC = 1;
+    DSC = 2;
 }
 
 enum CompressType {
-  COMP_NONE = 0;
-  NULL_SUPPRESS = 1;
-  RUN_LENGTH = 2;
-  BIT_VECTOR = 3;
-  DICTIONARY = 4;
-  SNAPPY = 5;
-  LZ = 6;
+    COMP_NONE = 0;
+    NULL_SUPPRESS = 1;
+    RUN_LENGTH = 2;
+    BIT_VECTOR = 3;
+    DICTIONARY = 4;
+    SNAPPY = 5;
+    LZ = 6;
 }
 
 message ColumnMetaProto {
-  required DataType dataType = 1;
-  required bool compressed = 2;
-  required bool sorted = 3;
-  required bool contiguous = 4;
-  required StoreType storeType = 5;
-  required CompressType compType = 6;
-  required int64 startRid = 7;
-  required int32 recordNum = 8;
-  required int32 offsetToIndex = 9;
+    required DataType dataType = 1;
+    required bool compressed = 2;
+    required bool sorted = 3;
+    required bool contiguous = 4;
+    required StoreType storeType = 5;
+    required CompressType compType = 6;
+    required int64 startRid = 7;
+    required int32 recordNum = 8;
+    required int32 offsetToIndex = 9;
 }
 
 message ColumnProto {
-  required string columnName = 1;
-  required DataType dataType = 2;
+	required string columnName = 1;
+	required DataType dataType = 2;
 }
 
 message SchemaProto {
-  repeated ColumnProto fields = 1;
+	repeated ColumnProto fields = 1;
 }
 
 message KeyValueProto {
-  required string key = 1;
-  required string value = 2;
+	required string key = 1;
+	required string value = 2;
 }
 
 message KeyValueSetProto {
-  repeated KeyValueProto keyval = 1;
+	repeated KeyValueProto keyval = 1;
 }
 
 message FragmentProto {
-  required string id = 1;
-  required string path = 2;
-  required int64 startOffset = 3;
-  required int64 length = 4;
-  required TableProto meta = 5;
-  optional TableStatProto stat = 6;
+	required string id = 1;
+	required string path = 2;
+	required int64 startOffset = 3;
+	required int64 length = 4;
+	required TableProto meta = 5;
+	optional TableStatProto stat = 6;
   optional bool distCached = 7 [default = false];
 }
 
 message TableProto {
-  required SchemaProto schema = 1;
-  required StoreType storeType = 2;
-  required KeyValueSetProto params = 3;
-  optional TableStatProto stat = 4;
+    required SchemaProto schema = 1;
+    required StoreType storeType = 2;
+    required KeyValueSetProto params = 3;
+    optional TableStatProto stat = 4;
 }
 
 message TableDescProto {
-  required string id = 1;
-  required string path = 2;
-  required TableProto meta = 3;
+	required string id = 1;
+	required string path = 2;
+	required TableProto meta = 3;
 }
 
 enum FunctionType {
-  GENERAL = 0;
-  AGGREGATION = 1;
+	GENERAL = 0;
+	AGGREGATION = 1;
 }
 
 message FunctionDescProto {
-  required string signature = 1;
-  required string className = 2;
-  required FunctionType type = 3;
-  repeated DataType parameterTypes = 4;
-  required DataType returnType = 5;
+	required string signature = 1;
+	required string className = 2;
+	required FunctionType type = 3;
+	repeated DataType parameterTypes = 4;
+	required DataType returnType = 5;
 }
 
 message IndexDescProto {
-  required string name = 1;
-  required string tableId = 2;
-  required ColumnProto column = 3;
-  required IndexMethod indexMethod = 4;
-  optional bool isUnique = 5 [default = false];
-  optional bool isClustered = 6 [default = false];
-  optional bool isAscending = 7 [default = false];
+    required string name = 1;
+    required string tableId = 2;
+    required ColumnProto column = 3;
+    required IndexMethod indexMethod = 4;
+    optional bool isUnique = 5 [default = false];
+    optional bool isClustered = 6 [default = false];
+    optional bool isAscending = 7 [default = false];
 }
 
 enum IndexMethod {
-  TWO_LEVEL_BIN_TREE = 0;
-  BTREE = 1;
-  HASH = 2;
-  BITMAP = 3;
+    TWO_LEVEL_BIN_TREE = 0;
+    BTREE = 1;
+    HASH = 2;
+    BITMAP = 3;
 }
 
 message GetAllTableNamesResponse {
-  repeated string tableName = 1;
+    repeated string tableName = 1;
 }
 
 message GetIndexRequest {
-  required string tableName = 1;
-  required string columnName = 2;
+    required string tableName = 1;
+    required string columnName = 2;
 }
 
 message GetFunctionsResponse {
-  repeated FunctionDescProto functionDesc = 1;
+	repeated FunctionDescProto functionDesc = 1;
 }
 
 message UnregisterFunctionRequest {
-  required string signature = 1;
-  repeated DataType parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message GetFunctionMetaRequest {
-  required string signature = 1;
-  repeated DataType parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message ContainFunctionRequest {
-  required string signature = 1;
-  repeated DataType parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message TableStatProto {
-  required int64 numRows = 1;
-  required int64 numBytes = 2;
-  optional int32 numBlocks = 3;
-  optional int32 numPartitions = 4;
-  optional int64 avgRows = 5;
-  repeated ColumnStatProto colStat = 6;
+	required int64 numRows = 1;
+	required int64 numBytes = 2;
+	optional int32 numBlocks = 3;
+	optional int32 numPartitions = 4;
+	optional int64 avgRows = 5;
+	repeated ColumnStatProto colStat = 6;
 }
 
 message ColumnStatProto {
-  required ColumnProto column = 1;
-  optional int64 numDistVal = 2;
-  optional int64 numNulls = 3;
-  optional bytes minValue = 4;
-  optional bytes maxValue = 5;
+    required ColumnProto column = 1;
+    optional int64 numDistVal = 2;
+    optional int64 numNulls = 3;
+    optional bytes minValue = 4;
+    optional bytes maxValue = 5;
 }
 
 enum StatType {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
index 936e8c1..1ede050 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
@@ -22,7 +22,6 @@ import org.apache.tajo.catalog.function.GeneralFunction;
 import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
-import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.DatumFactory;
@@ -45,8 +44,8 @@ public class TestFunctionDesc {
     private Integer y;
 
     public TestSum() {
-      super(new Column[] { new Column("arg1", TajoDataTypes.Type.INT4),
-          new Column("arg2", TajoDataTypes.Type.INT4) });
+      super(new Column[] { new Column("arg1", org.apache.tajo.common.TajoDataTypes.Type.INT4),
+          new Column("arg2", org.apache.tajo.common.TajoDataTypes.Type.INT4) });
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 c3d0d4a..e82afc8 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
@@ -134,7 +134,7 @@ public class CatalogServer extends AbstractService {
       this.rpcServer.start();
 
       this.bindAddress = this.rpcServer.getBindAddress();
-      this.serverName = org.apache.tajo.util.NetUtils.getIpPortString(bindAddress);
+      this.serverName = NetUtils.getIpPortString(bindAddress);
       conf.setVar(ConfVars.CATALOG_ADDRESS, serverName);
     } catch (Exception e) {
       LOG.error("Cannot start RPC Server of CatalogServer", e);
@@ -145,7 +145,9 @@ public class CatalogServer extends AbstractService {
   }
 
   public void stop() {
-    this.rpcServer.shutdown();
+    if (rpcServer != null) {
+      this.rpcServer.shutdown();
+    }
     LOG.info("Catalog Server (" + serverName + ") shutdown");
     super.stop();
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 4394d74..5dbbaca 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryId.java
@@ -18,7 +18,9 @@
 
 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;
@@ -112,7 +114,7 @@ public class QueryId implements Comparable<QueryId> {
     return new ApplicationIdPBImpl(p);
   }
 
-  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+  public static ApplicationIdProto convertToProtoFormat(ApplicationId t) {
     return ((ApplicationIdPBImpl)t).getProto();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 5ad16a2..fb7c268 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,6 +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"),
 
     //////////////////////////////////
     // Catalog Configuration

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-common/src/main/java/org/apache/tajo/util/Bytes.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/Bytes.java b/tajo-common/src/main/java/org/apache/tajo/util/Bytes.java
index 64ba1b8..33dd9d9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/Bytes.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/Bytes.java
@@ -133,7 +133,7 @@ public class Bytes {
    * Read byte-array written with a WritableableUtils.vint prefix.
    * @param in Input to read from.
    * @return byte array read off <code>in</code>
-   * @throws IOException e
+   * @throws java.io.IOException e
    */
   public static byte [] readByteArray(final DataInput in)
   throws IOException {
@@ -164,7 +164,7 @@ public class Bytes {
    * Write byte-array with a WritableableUtils.vint prefix.
    * @param out output stream to be written to
    * @param b array to write
-   * @throws IOException e
+   * @throws java.io.IOException e
    */
   public static void writeByteArray(final DataOutput out, final byte [] b)
   throws IOException {
@@ -181,7 +181,7 @@ public class Bytes {
    * @param b array
    * @param offset offset into array
    * @param length length past offset
-   * @throws IOException e
+   * @throws java.io.IOException e
    */
   public static void writeByteArray(final DataOutput out, final byte [] b,
       final int offset, final int length)
@@ -975,7 +975,7 @@ public class Bytes {
  
   interface Comparer<T> {
     abstract public int compareTo(T buffer1, int offset1, int length1,
-        T buffer2, int offset2, int length2);
+                                  T buffer2, int offset2, int length2);
   }
 
   @VisibleForTesting
@@ -985,7 +985,7 @@ public class Bytes {
 
   /**
    * Provides a lexicographical comparer implementation; either a Java
-   * implementation or a faster implementation based on {@link Unsafe}.
+   * implementation or a faster implementation based on {@link sun.misc.Unsafe}.
    *
    * <p>Uses reflection to gracefully fall back to the Java implementation if
    * {@code Unsafe} isn't available.
@@ -1224,7 +1224,7 @@ public class Bytes {
 
   /**
    * @param b bytes to hash
-   * @return Runs {@link WritableComparator#hashBytes(byte[], int)} on the
+   * @return Runs {@link org.apache.hadoop.io.WritableComparator#hashBytes(byte[], int)} on the
    * passed in array.  This method is what {@link org.apache.hadoop.io.Text} and
    * {@link ImmutableBytesWritable} use calculating hash code.
    */
@@ -1235,7 +1235,7 @@ public class Bytes {
   /**
    * @param b value
    * @param length length of the value
-   * @return Runs {@link WritableComparator#hashBytes(byte[], int)} on the
+   * @return Runs {@link org.apache.hadoop.io.WritableComparator#hashBytes(byte[], int)} on the
    * passed in array.  This method is what {@link org.apache.hadoop.io.Text} and
    * {@link ImmutableBytesWritable} use calculating hash code.
    */

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
index 7ff1c1f..f045197 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/CommonTestingUtil.java
@@ -30,7 +30,7 @@ public class CommonTestingUtil {
    *
    * @param dir a local directory to be created
    * @return  the created path
-   * @throws IOException
+   * @throws java.io.IOException
    */
   public static Path getTestDir(String dir) throws IOException {
     Path path = new Path(dir);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 2164d27..9dfbfbc 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
@@ -21,11 +21,20 @@ 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.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;
+
 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);
@@ -45,7 +54,7 @@ public class TajoIdUtils {
     String[] split = queryId.split(QueryId.SEPARATOR);
     ApplicationId appId = BuilderUtils.newApplicationId(Long.valueOf(split[1]),
         Integer.parseInt(split[2]));
-    int idInt = Integer.parseInt(split[2]);
+    int idInt = Integer.parseInt(split[3]);
     return newQueryId(appId, idInt);
   }
 
@@ -80,4 +89,33 @@ public class TajoIdUtils {
     SubQueryId subId = new SubQueryId(proto);
     return subId;
   }
+
+  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);
+    }
+  }
+
+  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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 99bb666..541b4d4 100644
--- a/tajo-core/tajo-core-backend/pom.xml
+++ b/tajo-core/tajo-core-backend/pom.xml
@@ -173,9 +173,11 @@
                 <argument>-Isrc/main/proto/</argument>
                 <argument>--java_out=target/generated-sources/proto</argument>
                 <argument>src/main/proto/tajo_protos.proto</argument>
-                <argument>src/main/proto/MasterWorkerProtos.proto</argument>
-                <argument>src/main/proto/MasterWorkerProtocol.proto</argument>
-                <argument>src/main/proto/ClientProtocol.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/TajoMasterClientProtocol.proto</argument>
               </arguments>
             </configuration>
             <goals>
@@ -365,10 +367,17 @@
       <artifactId>commons-lang</artifactId>
       <version>2.6</version>
     </dependency>
+    <!--
     <dependency>
       <groupId>org.jboss.netty</groupId>
       <artifactId>netty</artifactId>
     </dependency>
+    -->
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty</artifactId>
+    </dependency>
+
     <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
@@ -407,6 +416,14 @@
               </execution>
             </executions>
           </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-report-plugin</artifactId>
+            <version>2.15</version>
+            <configuration>
+              <aggregate>true</aggregate>
+            </configuration>
+          </plugin>
         </plugins>
       </build>
     </profile>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
index 42717d0..1664b1c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/benchmark/BenchmarkSet.java
@@ -52,8 +52,17 @@ public abstract class BenchmarkSet {
   }
 
   protected void loadQueries(String dir) throws IOException {
+    // TODO - this code dead??
     File queryDir = new File(dir);
 
+    if(!queryDir.exists()) {
+      queryDir = new File(System.getProperty("user.dir") + "/tajo-core/tajo-core-backend/" + dir);
+    }
+
+    if(!queryDir.exists())
+    {
+      return;
+    }
     int last;
     String name, query;
     for (String file : queryDir.list()) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 ce4c72f..86d47f9 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
@@ -28,12 +28,11 @@ import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.Column;
 import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.client.ClientProtocol;
 import org.apache.tajo.client.QueryStatus;
 import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.master.cluster.ServerName;
+import org.apache.tajo.ipc.ClientProtos;
 import org.apache.tajo.util.FileUtil;
 import org.apache.tajo.util.TajoIdUtils;
 
@@ -265,7 +264,6 @@ public class TajoCli {
   }
 
   public int executeStatements(String line) throws Exception {
-
     String stripped;
     for (String statement : line.split(";")) {
       stripped = StringUtils.chomp(statement);
@@ -284,14 +282,20 @@ public class TajoCli {
         invokeCommand(cmds);
 
       } else { // submit a query to TajoMaster
-        ClientProtocol.SubmitQueryRespose response = client.executeQuery(stripped);
-
-        if (response.getResultCode() == ClientProtocol.ResultCode.OK) {
-          QueryId queryId = new QueryId(response.getQueryId());
-          if (queryId.equals(TajoIdUtils.NullQueryId)) {
-            sout.println("OK");
-          } else {
-            getQueryResult(queryId);
+        ClientProtos.SubmitQueryResponse response = client.executeQuery(stripped);
+        if (response.getResultCode() == ClientProtos.ResultCode.OK) {
+          QueryId queryId = null;
+          try {
+            queryId = new QueryId(response.getQueryId());
+            if (queryId.equals(TajoIdUtils.NullQueryId)) {
+              sout.println("OK");
+            } else {
+              getQueryResult(queryId);
+            }
+          } finally {
+            if(queryId != null) {
+              client.closeQuery(queryId);
+            }
           }
         } else {
         if (response.hasErrorMessage()) {
@@ -318,8 +322,13 @@ public class TajoCli {
 
       QueryStatus status;
       while (true) {
+        // TODO - configurable
         Thread.sleep(1000);
         status = client.getQueryStatus(queryId);
+        if(status.getState() == QueryState.QUERY_MASTER_INIT || status.getState() == QueryState.QUERY_MASTER_LAUNCHED) {
+          continue;
+        }
+
         if (status.getState() == QueryState.QUERY_RUNNING ||
             status.getState() == QueryState.QUERY_SUCCEEDED) {
           sout.println("Progress: " + (int)(status.getProgress() * 100.0f)
@@ -328,7 +337,7 @@ public class TajoCli {
           sout.flush();
         }
 
-        if (status.getState() != QueryState.QUERY_RUNNING) {
+        if (status.getState() != QueryState.QUERY_RUNNING && status.getState() != QueryState.QUERY_NOT_ASSIGNED) {
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
index cdde9de..c7122b3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/QueryStatus.java
@@ -20,7 +20,7 @@ package org.apache.tajo.client;
 
 import org.apache.tajo.QueryId;
 import org.apache.tajo.TajoProtos.QueryState;
-import org.apache.tajo.client.ClientProtocol.GetQueryStatusResponse;
+import org.apache.tajo.ipc.ClientProtos.GetQueryStatusResponse;
 
 public class QueryStatus {
   private QueryId queryId;
@@ -31,6 +31,8 @@ public class QueryStatus {
   private long finishTime;
   private boolean hasResult;
   private String errorText;
+  private String queryMasterHost;
+  private int queryMasterPort;
 
   public QueryStatus(GetQueryStatusResponse proto) {
     queryId = new QueryId(proto.getQueryId());
@@ -43,6 +45,17 @@ public class QueryStatus {
     if (proto.hasErrorMessage()) {
       errorText = proto.getErrorMessage();
     }
+
+    queryMasterHost = proto.getQueryMasterHost();
+    queryMasterPort = proto.getQueryMasterPort();
+  }
+
+  public String getQueryMasterHost() {
+    return queryMasterHost;
+  }
+
+  public int getQueryMasterPort() {
+    return queryMasterPort;
   }
 
   public QueryId getQueryId() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 8317065..5b4b064 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
@@ -28,10 +28,15 @@ import org.apache.tajo.TajoProtos.QueryState;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.TableDesc;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.client.ClientProtocol.*;
 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.TajoMasterClientProtocol;
+import org.apache.tajo.ipc.TajoMasterClientProtocol.*;
+import org.apache.tajo.ipc.QueryMasterClientProtocol;
+import org.apache.tajo.ipc.QueryMasterClientProtocol.*;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.ClientProtos.*;
 import org.apache.tajo.rpc.ProtoBlockingRpcClient;
 import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
 import org.apache.tajo.util.TajoIdUtils;
@@ -39,14 +44,25 @@ import org.apache.tajo.util.TajoIdUtils;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.sql.ResultSet;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
+/**
+ * TajoClient is ThreadSafe
+ */
 public class TajoClient {
   private final Log LOG = LogFactory.getLog(TajoClient.class);
 
   private final TajoConf conf;
-  private ProtoBlockingRpcClient client;
-  private ClientProtocolService.BlockingInterface service;
+  private ProtoBlockingRpcClient tasjoMasterClient;
+  private TajoMasterClientProtocolService.BlockingInterface tajoMasterService;
+
+  private Map<QueryId, QueryMasterClientProtocolService.BlockingInterface> queryMasterConnectionMap =
+          new HashMap<QueryId, QueryMasterClientProtocolService.BlockingInterface>();
+
+  private Map<QueryId, ProtoBlockingRpcClient> queryMasterClientMap =
+          new HashMap<QueryId, ProtoBlockingRpcClient>();
 
   public TajoClient(TajoConf conf) throws IOException {
     this.conf = conf;
@@ -67,8 +83,8 @@ public class TajoClient {
 
   private void connect(InetSocketAddress addr) throws IOException {
     try {
-      client = new ProtoBlockingRpcClient(ClientProtocol.class, addr);
-      service = client.getStub();
+      tasjoMasterClient = new ProtoBlockingRpcClient(TajoMasterClientProtocol.class, addr);
+      tajoMasterService = tasjoMasterClient.getStub();
     } catch (Exception e) {
       throw new IOException(e);
     }
@@ -78,11 +94,31 @@ public class TajoClient {
   }
 
   public void close() {
-    client.close();
+    tasjoMasterClient.close();
+
+    for(ProtoBlockingRpcClient eachClient: queryMasterClientMap.values()) {
+      eachClient.close();
+    }
+    queryMasterClientMap.clear();
+    queryMasterConnectionMap.clear();
+  }
+
+  public void closeQuery(QueryId queryId) {
+    if(queryMasterClientMap.containsKey(queryId)) {
+      try {
+        queryMasterConnectionMap.get(queryId).killQuery(null, queryId.getProto());
+      } catch (Exception e) {
+        LOG.warn("Fail to close query:" + queryId + "," + e.getMessage(), e);
+      }
+      queryMasterClientMap.get(queryId).close();
+      LOG.info("Closed QueryMaster connection(" + queryId + "," + queryMasterClientMap.get(queryId).getRemoteAddress() + ")");
+      queryMasterClientMap.remove(queryId);
+      queryMasterConnectionMap.remove(queryId);
+    }
   }
 
   public boolean isConnected() {
-    return client.isConnected();
+    return tasjoMasterClient.isConnected();
   }
 
   /**
@@ -91,11 +127,11 @@ 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 SubmitQueryRespose executeQuery(String tql) throws ServiceException {
+  public ClientProtos.SubmitQueryResponse executeQuery(String tql) throws ServiceException {
     QueryRequest.Builder builder = QueryRequest.newBuilder();
     builder.setQuery(tql);
 
-    return service.submitQuery(null, builder.build());
+    return tajoMasterService.submitQuery(null, builder.build());
   }
 
   /**
@@ -110,15 +146,12 @@ public class TajoClient {
       throws ServiceException, IOException {
     QueryRequest.Builder builder = QueryRequest.newBuilder();
     builder.setQuery(sql);
-    SubmitQueryRespose response = service.submitQuery(null, builder.build());
-    if (response.getResultCode() == ResultCode.OK) {
-      QueryId queryId = new QueryId(response.getQueryId());
-      if (queryId.equals(TajoIdUtils.NullQueryId)) {
-        return null;
-      }
-      return getQueryResultAndWait(queryId);
+    SubmitQueryResponse response = tajoMasterService.submitQuery(null, builder.build());
+    QueryId queryId = new QueryId(response.getQueryId());
+    if (queryId.equals(TajoIdUtils.NullQueryId)) {
+      return this.createNullResultSet(queryId);
     } else {
-      throw new ServiceException(response.getErrorMessage());
+      return this.getQueryResultAndWait(queryId);
     }
   }
 
@@ -127,38 +160,69 @@ public class TajoClient {
         = GetQueryStatusRequest.newBuilder();
     builder.setQueryId(queryId.getProto());
 
-    GetQueryStatusResponse res = service.getQueryStatus(null,
-        builder.build());
+    GetQueryStatusResponse res = null;
+    if(queryMasterConnectionMap.containsKey(queryId)) {
+      QueryMasterClientProtocolService.BlockingInterface queryMasterService = queryMasterConnectionMap.get(queryId);
+      res = queryMasterService.getQueryStatus(null, builder.build());
+    } else {
+      res = tajoMasterService.getQueryStatus(null, builder.build());
 
+      String queryMasterHost = res.getQueryMasterHost();
+      if(queryMasterHost != null && !queryMasterHost.isEmpty()) {
+        LOG.info("=========> connect to querymaster:" + queryMasterHost);
+        connectionToQueryMaster(queryId, queryMasterHost, res.getQueryMasterPort());
+      }
+    }
     return new QueryStatus(res);
   }
 
+  private void connectionToQueryMaster(QueryId queryId, String queryMasterHost, int queryMasterPort) {
+    try {
+      InetSocketAddress addr = NetUtils.createSocketAddr(queryMasterHost, queryMasterPort);
+      ProtoBlockingRpcClient client = new ProtoBlockingRpcClient(QueryMasterClientProtocol.class, addr);
+      QueryMasterClientProtocolService.BlockingInterface service = client.getStub();
+
+      queryMasterConnectionMap.put(queryId, service);
+      queryMasterClientMap.put(queryId, client);
+
+      LOG.debug("connected to Query Master (" +
+              org.apache.tajo.util.NetUtils.getIpPortString(addr) + ")");
+    } catch (Exception e) {
+      LOG.error(e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
   private static boolean isQueryRunnning(QueryState state) {
     return state == QueryState.QUERY_NEW ||
         state == QueryState.QUERY_INIT ||
-        state == QueryState.QUERY_RUNNING;
+        state == QueryState.QUERY_RUNNING ||
+        state == QueryState.QUERY_MASTER_LAUNCHED ||
+        state == QueryState.QUERY_MASTER_INIT ||
+        state == QueryState.QUERY_NOT_ASSIGNED;
   }
 
   public ResultSet getQueryResult(QueryId queryId)
       throws ServiceException, IOException {
-    if (queryId.equals(TajoIdUtils.NullQueryId)) {
-      return null;
-    }
+      if (queryId.equals(TajoIdUtils.NullQueryId)) {
+        return createNullResultSet(queryId);
+      }
 
     TableDesc tableDesc = getResultDesc(queryId);
-    return new ResultSetImpl(conf, tableDesc.getPath());
+    return new ResultSetImpl(this, queryId, conf, tableDesc.getPath());
   }
 
   public ResultSet getQueryResultAndWait(QueryId queryId)
       throws ServiceException, IOException {
     if (queryId.equals(TajoIdUtils.NullQueryId)) {
-      return null;
+      return createNullResultSet(queryId);
     }
     QueryStatus status = getQueryStatus(queryId);
 
     while(status != null && isQueryRunnning(status.getState())) {
       try {
-        Thread.sleep(500);
+//        Thread.sleep(500);
+        Thread.sleep(2000);
       } catch (InterruptedException e) {
         e.printStackTrace();
       }
@@ -170,24 +234,34 @@ public class TajoClient {
       if (status.hasResult()) {
         return getQueryResult(queryId);
       } else {
-        return null;
+        return createNullResultSet(queryId);
       }
 
     } else {
-      LOG.error(status.getErrorMessage());
+      LOG.warn("=====>Query failed:" + status.getState());
 
-      return null;
+      //TODO throw SQLException(?)
+      return createNullResultSet(queryId);
     }
   }
 
+  public ResultSet createNullResultSet(QueryId queryId) throws IOException {
+    return new ResultSetImpl(this, queryId);
+  }
+
   public TableDesc getResultDesc(QueryId queryId) throws ServiceException {
     if (queryId.equals(TajoIdUtils.NullQueryId)) {
       return null;
     }
 
+    QueryMasterClientProtocolService.BlockingInterface queryMasterService = queryMasterConnectionMap.get(queryId);
+    if(queryMasterService == null) {
+      LOG.warn("No Connection to QueryMaster for " + queryId);
+      return null;
+    }
     GetQueryResultRequest.Builder builder = GetQueryResultRequest.newBuilder();
     builder.setQueryId(queryId.getProto());
-    GetQueryResultResponse response = service.getQueryResult(null,
+    GetQueryResultResponse response = queryMasterService.getQueryResult(null,
         builder.build());
 
     return CatalogUtil.newTableDesc(response.getTableDesc());
@@ -198,14 +272,14 @@ public class TajoClient {
     builder.setQuery(tql);
 
     ResultCode resultCode =
-        service.updateQuery(null, builder.build()).getResultCode();
+        tajoMasterService.updateQuery(null, builder.build()).getResultCode();
     return resultCode == ResultCode.OK;
   }
 
   public boolean existTable(String name) throws ServiceException {
     StringProto.Builder builder = StringProto.newBuilder();
     builder.setValue(name);
-    return service.existTable(null, builder.build()).getValue();
+    return tajoMasterService.existTable(null, builder.build()).getValue();
   }
 
   public TableDesc attachTable(String name, String path)
@@ -213,7 +287,7 @@ public class TajoClient {
     AttachTableRequest.Builder builder = AttachTableRequest.newBuilder();
     builder.setName(name);
     builder.setPath(path);
-    TableResponse res = service.attachTable(null, builder.build());
+    TableResponse res = tajoMasterService.attachTable(null, builder.build());
     return CatalogUtil.newTableDesc(res.getTableDesc());
   }
 
@@ -225,7 +299,7 @@ public class TajoClient {
   public boolean detachTable(String name) throws ServiceException {
     StringProto.Builder builder = StringProto.newBuilder();
     builder.setValue(name);
-    return service.detachTable(null, builder.build()).getValue();
+    return tajoMasterService.detachTable(null, builder.build()).getValue();
   }
 
   public TableDesc createTable(String name, Path path, TableMeta meta)
@@ -234,14 +308,14 @@ public class TajoClient {
     builder.setName(name);
     builder.setPath(path.toString());
     builder.setMeta(meta.getProto());
-    TableResponse res = service.createTable(null, builder.build());
+    TableResponse res = tajoMasterService.createTable(null, builder.build());
     return CatalogUtil.newTableDesc(res.getTableDesc());
   }
 
   public boolean dropTable(String name) throws ServiceException {
     StringProto.Builder builder = StringProto.newBuilder();
     builder.setValue(name);
-    return service.dropTable(null, builder.build()).getValue();
+    return tajoMasterService.dropTable(null, builder.build()).getValue();
   }
 
   /**
@@ -250,14 +324,14 @@ public class TajoClient {
    */
   public List<String> getTableList() throws ServiceException {
     GetTableListRequest.Builder builder = GetTableListRequest.newBuilder();
-    GetTableListResponse res = service.getTableList(null, builder.build());
+    GetTableListResponse res = tajoMasterService.getTableList(null, builder.build());
     return res.getTablesList();
   }
 
   public TableDesc getTableDesc(String tableName) throws ServiceException {
     GetTableDescRequest.Builder build = GetTableDescRequest.newBuilder();
     build.setTableName(tableName);
-    TableResponse res = service.getTableDesc(null, build.build());
+    TableResponse res = tajoMasterService.getTableDesc(null, build.build());
     if (res == null) {
       return null;
     } else {
@@ -272,7 +346,7 @@ public class TajoClient {
 
     try {
       /* send a kill to the TM */
-      service.killQuery(null, queryId.getProto());
+      tajoMasterService.killQuery(null, queryId.getProto());
       long currentTimeMillis = System.currentTimeMillis();
       long timeKillIssued = currentTimeMillis;
       while ((currentTimeMillis < timeKillIssued + 10000L) && (status.getState()
@@ -293,4 +367,14 @@ public class TajoClient {
 
     return true;
   }
+
+  public static void main(String[] args) throws Exception {
+    TajoClient client = new TajoClient(new TajoConf());
+
+    client.close();
+
+    synchronized(client) {
+      client.wait();
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
index bde2df5..973de99 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
@@ -98,7 +98,7 @@ public class EvalTreeUtil {
       found.add(node);
     }
   }
-  
+
   public static Schema getSchemaByTargets(Schema inputSchema, Target [] targets) 
       throws InternalException {
     Schema schema = new Schema();
@@ -282,7 +282,7 @@ public class EvalTreeUtil {
   }
   
   public static class VariableCounter implements EvalNodeVisitor {
-    private final Map<EvalNode.Type, Integer> counter;
+    private final Map<Type, Integer> counter;
     
     public VariableCounter() {
       counter = Maps.newHashMap();
@@ -299,7 +299,7 @@ public class EvalTreeUtil {
       }
     }
     
-    public Map<EvalNode.Type, Integer> getCounter() {
+    public Map<Type, Integer> getCounter() {
       return counter;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/FromTable.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/FromTable.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/FromTable.java
index 04c1bb9..0e3a174 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/FromTable.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/FromTable.java
@@ -41,6 +41,10 @@ public class FromTable implements Cloneable, GsonObject {
     this.alias = alias;
   }
 
+  public TableDesc getTableDesc() {
+    return this.desc;
+  }
+
   public final String getTableName() {
     return desc.getId();
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 bb7caa1..7d430c5 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,9 +19,7 @@
 package org.apache.tajo.engine.query;
 
 import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.engine.MasterWorkerProtos.Fetch;
-import org.apache.tajo.engine.MasterWorkerProtos.QueryUnitRequestProto;
-import org.apache.tajo.engine.MasterWorkerProtos.QueryUnitRequestProtoOrBuilder;
+import org.apache.tajo.ipc.QueryMasterProtocol.*;
 import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
 import org.apache.tajo.storage.Fragment;
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 db049a0..e0db8e1 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
@@ -24,9 +24,11 @@ package org.apache.tajo.engine.query;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
+import org.apache.tajo.QueryId;
 import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.TableMetaImpl;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableProto;
+import org.apache.tajo.client.TajoClient;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.datum.NullDatum;
 import org.apache.tajo.exception.UnsupportedException;
@@ -56,27 +58,36 @@ public class ResultSetImpl implements ResultSet {
   private int curRow;
   private long totalRow;
   private boolean wasNull;
+  private TajoClient tajoClient;
+  QueryId queryId;
 
-  public ResultSetImpl(Configuration conf, String path) throws IOException {
-    this(conf, new Path(path));
+  public ResultSetImpl(TajoClient tajoClient, QueryId queryId) throws IOException {
+    this(tajoClient, queryId, null, null);
   }
 
-  public ResultSetImpl(Configuration conf, Path path) throws IOException {
+//  public ResultSetImpl(TajoClient tajoClient, QueryId queryId, Configuration conf, String path) throws IOException {
+//    this(tajoClient, queryId, conf, new Path(path));
+//  }
+
+  public ResultSetImpl(TajoClient tajoClient, QueryId queryId, Configuration conf, Path path) throws IOException {
+    this.tajoClient = tajoClient;
+    this.queryId = queryId;
     this.conf = conf;
-    this.fs = path.getFileSystem(this.conf);
-    // TODO - to be improved. It can be solved to get the query finish status
-    // from master.
-    try {
-      this.meta = getMeta(this.conf, path);
-    } catch (FileNotFoundException fnf) {
-      this.totalRow = 0;
-      init();
-      return;
+    if(path != null) {
+      this.fs = path.getFileSystem(this.conf);
+      // TODO - to be improved. It can be solved to get the query finish status
+      // from master.
+      try {
+        this.meta = getMeta(this.conf, path);
+      } catch (FileNotFoundException fnf) {
+        this.totalRow = 0;
+        init();
+        return;
+      }
+      this.totalRow = meta.getStat() != null ? meta.getStat().getNumRows() : 0;
+      Collection<Fragment> frags = getFragmentsNG(meta, path);
+      scanner = new MergeScanner(conf, meta, frags);
     }
-    this.totalRow = meta.getStat() != null ? meta.getStat().getNumRows() : 0;
-    Collection<Fragment> frags = getFragmentsNG(meta, path);
-    scanner = new MergeScanner(conf, meta, frags);
-    scanner.init();
     init();
   }
 
@@ -194,7 +205,9 @@ public class ResultSetImpl implements ResultSet {
   @Override
   public void beforeFirst() throws SQLException {
     try {
-      scanner.reset();
+      if(scanner != null) {
+        scanner.reset();
+      }
       init();
     } catch (IOException e) {
       e.printStackTrace();
@@ -230,7 +243,17 @@ public class ResultSetImpl implements ResultSet {
   @Override
   public void close() throws SQLException {
     try {
-      this.scanner.close();
+      if(tajoClient != null) {
+        this.tajoClient.closeQuery(queryId);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+    try {
+      if(scanner != null) {
+        this.scanner.close();
+      }
+      //TODO clean temp result file
       cur = null;
       curRow = -1;
     } catch (IOException e) {
@@ -1232,6 +1255,9 @@ public class ResultSetImpl implements ResultSet {
    */
   @Override
   public boolean next() throws SQLException {
+    if(scanner == null) {
+      return false;
+    }
     try {
       if (totalRow <= 0)
         return false;
@@ -2220,4 +2246,8 @@ public class ResultSetImpl implements ResultSet {
   private void handleNull(Datum d) {
     wasNull = (d instanceof NullDatum);
   }
+
+  public boolean hasResult() {
+    return scanner != null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 880110d..bb4008f 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,14 +23,13 @@ package org.apache.tajo.ipc.protocolrecords;
 
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.engine.MasterWorkerProtos.Fetch;
-import org.apache.tajo.engine.MasterWorkerProtos.QueryUnitRequestProto;
+import org.apache.tajo.ipc.QueryMasterProtocol;
 import org.apache.tajo.storage.Fragment;
 
 import java.net.URI;
 import java.util.List;
 
-public interface QueryUnitRequest extends ProtoObject<QueryUnitRequestProto> {
+public interface QueryUnitRequest extends ProtoObject<QueryMasterProtocol.QueryUnitRequestProto> {
 
 	public QueryUnitAttemptId getId();
 	public List<Fragment> getFragments();
@@ -40,7 +39,7 @@ public interface QueryUnitRequest extends ProtoObject<QueryUnitRequestProto> {
 	public boolean isInterQuery();
 	public void setInterQuery();
 	public void addFetch(String name, URI uri);
-	public List<Fetch> getFetches();
+	public List<QueryMasterProtocol.Fetch> getFetches();
   public boolean shouldDie();
   public void setShouldDie();
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
deleted file mode 100644
index c980dcf..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ClientService.java
+++ /dev/null
@@ -1,411 +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 com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.commons.logging.Log;
-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.net.NetUtils;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.exception.AlreadyExistsTableException;
-import org.apache.tajo.catalog.exception.NoSuchTableException;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
-import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.client.ClientProtocol;
-import org.apache.tajo.client.ClientProtocol.*;
-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.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
-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.TajoIdUtils;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collection;
-
-public class ClientService extends AbstractService {
-  private final static Log LOG = LogFactory.getLog(ClientService.class);
-  private final MasterContext context;
-  private final TajoConf conf;
-  private final CatalogService catalog;
-  private final ClientProtocolHandler clientHandler;
-  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 ClientService(MasterContext context) {
-    super(ClientService.class.getName());
-    this.context = context;
-    this.conf = context.getConf();
-    this.catalog = context.getCatalog();
-    this.clientHandler = new ClientProtocolHandler();
-  }
-
-  @Override
-  public void start() {
-
-    // start the rpc server
-    String confClientServiceAddr = conf.getVar(ConfVars.CLIENT_SERVICE_ADDRESS);
-    InetSocketAddress initIsa = NetUtils.createSocketAddr(confClientServiceAddr);
-    try {
-      server = new ProtoBlockingRpcServer(ClientProtocol.class, clientHandler, initIsa);
-    } catch (Exception e) {
-      LOG.error(e);
-    }
-    server.start();
-    bindAddress = server.getBindAddress();
-    this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS,
-        org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
-    LOG.info("Instantiated ClientService at " + this.bindAddress);
-    super.start();
-  }
-
-  @Override
-  public void stop() {
-    server.shutdown();
-    super.stop();
-  }
-
-  public InetSocketAddress getBindAddress() {
-    return this.bindAddress;
-  }
-
-  public int getHttpPort() {
-    return 0;
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // ClientService
-  /////////////////////////////////////////////////////////////////////////////
-
-  public class ClientProtocolHandler implements ClientProtocolService.BlockingInterface {
-    @Override
-    public BoolProto updateSessionVariables(RpcController controller,
-                                            UpdateSessionVariableRequest request)
-        throws ServiceException {
-      return null;
-    }
-
-    @Override
-    public SubmitQueryRespose submitQuery(RpcController controller,
-                                          QueryRequest request)
-        throws ServiceException {
-
-      QueryId queryId;
-      SubmitQueryRespose.Builder build = SubmitQueryRespose.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.error(msg, e);
-        } else {
-          LOG.error(msg);
-        }
-        build.setErrorMessage(msg);
-        return build.build();
-      }
-
-      LOG.info("Query " + queryId + " is submitted");
-      build.setResultCode(ResultCode.OK);
-      build.setQueryId(queryId.getProto());
-
-      return build.build();
-    }
-
-    @Override
-    public UpdateQueryResponse updateQuery(RpcController controller,
-                                           QueryRequest request)
-        throws ServiceException {
-
-      UpdateQueryResponse.Builder builder = UpdateQueryResponse.newBuilder();
-      try {
-        context.getGlobalEngine().updateQuery(request.getQuery());
-        builder.setResultCode(ResultCode.OK);
-        return builder.build();
-      } catch (Exception e) {
-        builder.setResultCode(ResultCode.ERROR);
-        if (e.getMessage() == null) {
-          builder.setErrorMessage(ExceptionUtils.getStackTrace(e));
-        }
-        return builder.build();
-      }
-    }
-
-    @Override
-    public GetQueryResultResponse getQueryResult(RpcController controller,
-                                                 GetQueryResultRequest request)
-        throws ServiceException {
-      QueryId queryId = new QueryId(request.getQueryId());
-      if (queryId.equals(TajoIdUtils.NullQueryId)) {
-
-      }
-      Query query = context.getQuery(queryId).getContext().getQuery();
-
-      GetQueryResultResponse.Builder builder
-          = GetQueryResultResponse.newBuilder();
-      switch (query.getState()) {
-        case QUERY_SUCCEEDED:
-          builder.setTableDesc((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 GetQueryListResponse getQueryList(RpcController controller,
-                                             GetQueryListRequest request)
-        throws ServiceException {
-      return null;
-    }
-
-    @Override
-    public GetQueryStatusResponse getQueryStatus(RpcController controller,
-                                                 GetQueryStatusRequest request)
-        throws ServiceException {
-
-      GetQueryStatusResponse.Builder builder
-          = GetQueryStatusResponse.newBuilder();
-      QueryId queryId = new QueryId(request.getQueryId());
-      builder.setQueryId(request.getQueryId());
-
-      if (queryId.equals(TajoIdUtils.NullQueryId)) {
-        builder.setResultCode(ResultCode.OK);
-        builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
-      } else {
-        Query query = context.getQuery(queryId).getContext().getQuery();
-        if (query != null) {
-          builder.setResultCode(ResultCode.OK);
-          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.setResultCode(ResultCode.ERROR);
-          builder.setErrorMessage("No such query: " + queryId.toString());
-        }
-      }
-
-      return builder.build();
-    }
-
-    @Override
-    public BoolProto killQuery(RpcController controller,
-                               ApplicationAttemptIdProto request)
-        throws ServiceException {
-      QueryId queryId = new QueryId(request);
-      QueryMaster query = context.getQuery(queryId);
-      query.handle(new QueryEvent(queryId, QueryEventType.KILL));
-
-      return BOOL_TRUE;
-    }
-
-    @Override
-    public GetClusterInfoResponse getClusterInfo(RpcController controller,
-                                                 GetClusterInfoRequest request)
-        throws ServiceException {
-      return null;
-    }
-
-    @Override
-    public BoolProto existTable(RpcController controller,
-                                StringProto tableNameProto)
-        throws ServiceException {
-      String tableName = tableNameProto.getValue();
-      if (catalog.existsTable(tableName)) {
-        return BOOL_TRUE;
-      } else {
-        return BOOL_FALSE;
-      }
-    }
-
-    @Override
-    public GetTableListResponse getTableList(RpcController controller,
-                                             GetTableListRequest request)
-        throws ServiceException {
-      Collection<String> tableNames = catalog.getAllTableNames();
-      GetTableListResponse.Builder builder = GetTableListResponse.newBuilder();
-      builder.addAllTables(tableNames);
-      return builder.build();
-    }
-
-    @Override
-    public TableResponse getTableDesc(RpcController controller,
-                                      GetTableDescRequest request)
-        throws ServiceException {
-      String name = request.getTableName();
-      if (catalog.existsTable(name)) {
-        return TableResponse.newBuilder()
-            .setTableDesc((TableDescProto) catalog.getTableDesc(name).getProto())
-            .build();
-      } else {
-        return null;
-      }
-    }
-
-    @Override
-    public TableResponse createTable(RpcController controller, CreateTableRequest request)
-        throws ServiceException {
-      Path path = new Path(request.getPath());
-      TableMeta meta = new TableMetaImpl(request.getMeta());
-      TableDesc desc;
-      try {
-        desc = context.getGlobalEngine().createTable(request.getName(), meta, path);
-      } catch (Exception e) {
-        return TableResponse.newBuilder().setErrorMessage(e.getMessage()).build();
-      }
-
-      return TableResponse.newBuilder().setTableDesc((TableDescProto) desc.getProto()).build();
-    }
-
-    @Override
-    public BoolProto dropTable(RpcController controller,
-                               StringProto tableNameProto)
-        throws ServiceException {
-      context.getGlobalEngine().dropTable(tableNameProto.getValue());
-      return BOOL_TRUE;
-    }
-
-    @Override
-    public TableResponse attachTable(RpcController controller,
-                                     AttachTableRequest request)
-        throws ServiceException {
-
-      TableDesc desc;
-      if (catalog.existsTable(request.getName())) {
-        throw new AlreadyExistsTableException(request.getName());
-      }
-
-      Path path = new Path(request.getPath());
-
-      LOG.info(path.toUri());
-
-      TableMeta meta;
-      try {
-        meta = TableUtil.getTableMeta(conf, path);
-      } catch (IOException e) {
-        throw new RemoteException(e);
-      }
-
-      FileSystem fs;
-
-      // for legacy table structure
-      Path tablePath = new Path(path, "data");
-      try {
-        fs = path.getFileSystem(conf);
-        if (!fs.exists(tablePath)) {
-          tablePath = path;
-        }
-      } catch (IOException e) {
-        LOG.error(e);
-        return null;
-      }
-
-      if (meta.getStat() == null) {
-        long totalSize = 0;
-        try {
-          totalSize = calculateSize(tablePath);
-        } catch (IOException e) {
-          LOG.error("Cannot calculate the size of the relation", e);
-          return null;
-        }
-
-        meta = new TableMetaImpl(meta.getProto());
-        TableStat stat = new TableStat();
-        stat.setNumBytes(totalSize);
-        meta.setStat(stat);
-      }
-
-      desc = new TableDescImpl(request.getName(), meta, path);
-      catalog.addTable(desc);
-      LOG.info("Table " + desc.getId() + " is attached ("
-          + meta.getStat().getNumBytes() + ")");
-
-      return TableResponse.newBuilder().
-          setTableDesc((TableDescProto) desc.getProto())
-          .build();
-    }
-
-    @Override
-    public BoolProto detachTable(RpcController controller,
-                                 StringProto tableNameProto)
-        throws ServiceException {
-      String tableName = tableNameProto.getValue();
-      if (!catalog.existsTable(tableName)) {
-        throw new NoSuchTableException(tableName);
-      }
-
-      catalog.deleteTable(tableName);
-
-      LOG.info("Table " + tableName + " is detached");
-      return BOOL_TRUE;
-    }
-  }
-
-  private long calculateSize(Path path) throws IOException {
-    FileSystem fs = path.getFileSystem(conf);
-    long totalSize = 0;
-    for (FileStatus status : fs.listStatus(path)) {
-      totalSize += status.getLen();
-    }
-
-    return totalSize;
-  }
-}


[4/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..871ba77
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -0,0 +1,817 @@
+/**
+ * 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.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.net.NetUtils;
+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.engine.planner.global.GlobalOptimizer;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.ExprType;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+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.storage.StorageManager;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+// TODO - when exception, send error status to QueryMasterManager
+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;
+
+  //service handler for QueryMasterManager, Worker
+  private QueryMasterService queryMasterService;
+  private QueryMasterClientService queryMasterClientService;
+
+  private TaskRunnerLauncher taskRunnerLauncher;
+  private GlobalPlanner globalPlanner;
+  private GlobalOptimizer globalOptimizer;
+
+  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 String queryMasterManagerAddress;
+
+  private YarnRPC yarnRPC;
+
+  private YarnClient yarnClient;
+
+  private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
+
+  public QueryMaster(final QueryId queryId, final long appSubmitTime, String queryMasterManagerAddress) {
+    super(QueryMaster.class.getName());
+
+    this.queryId = queryId;
+    this.appSubmitTime = appSubmitTime;
+    this.appId = queryId.getApplicationId();
+    this.queryMasterManagerAddress = queryMasterManagerAddress;
+
+    LOG.info("Created Query Master for " + queryId);
+  }
+
+  public void init(Configuration conf) {
+    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();
+
+      LOG.info("Init QueryMasterManagerClient connection to:" + queryMasterManagerAddress);
+      InetSocketAddress addr = NetUtils.createSocketAddr(queryMasterManagerAddress);
+      queryMasterManagerClient = new ProtoBlockingRpcClient(QueryMasterManagerProtocol.class, addr);
+      queryMasterManagerService = queryMasterManagerClient.getStub();
+
+      clock = new SystemClock();
+
+      this.dispatcher = new AsyncDispatcher();
+      addIfService(dispatcher);
+
+      this.storageManager = new StorageManager(queryConf);
+
+      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());
+
+      clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
+
+      clientSessionTimeoutCheckThread.start();
+    } catch (Throwable t) {
+      LOG.error(t.getMessage(), t);
+      throw new RuntimeException(t);
+    }
+    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 = rpcServer.getBindAddress();
+        this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+      }
+      // Get the master address
+      LOG.info(QueryMasterService.class.getSimpleName() + " is bind to " + addr);
+      queryConf.setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
+    }
+
+    @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();
+      }
+      LOG.info("QueryMasterService stopped");
+      super.stop();
+    }
+
+    @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 done(RpcController controller, QueryMasterProtocol.TaskCompletionReport report,
+                     RpcCallback<PrimitiveProtos.BoolProto> done) {
+      queryContext.getEventHandler().handle(new TaskCompletionEvent(report));
+      done.run(TRUE_PROTO);
+    }
+
+    @Override
+    public void executeQuery(RpcController controller, PrimitiveProtos.StringProto request,
+                             RpcCallback<PrimitiveProtos.BoolProto> done) {
+    }
+  }
+
+  public void start() {
+    super.start();
+  }
+
+  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);
+      }
+    }
+
+    // TODO - release opened resource
+    if(this.queryMasterManagerClient != null) {
+      reportQueryStatus();
+
+      queryMasterManagerClient.close();
+    }
+
+    try {
+      FileSystem.closeAll();
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+    }
+
+    super.stop();
+
+    synchronized(queryId) {
+      queryId.notifyAll();
+    }
+  }
+
+  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);
+    }
+    try {
+      queryMasterManagerService.queryHeartbeat(null, queryHeartbeatBuilder.build());
+    } 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, ExprType.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);
+    }
+  }
+
+  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 QueryContext getContext() {
+    return this.queryContext;
+  }
+
+  public class QueryContext {
+    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) {
+      this.conf = conf;
+    }
+
+    public QueryConf getConf() {
+      return conf;
+    }
+
+    public InetSocketAddress getQueryMasterServiceAddress() {
+      return queryMasterService.bindAddr;
+    }
+
+    public QueryMasterClientService getQueryMasterClientService() {
+      return queryMasterClientService;
+    }
+
+    public AsyncDispatcher getDispatcher() {
+      return dispatcher;
+    }
+
+    public Clock getClock() {
+      return clock;
+    }
+
+    public Query getQuery() {
+      return query;
+    }
+
+    public SubQuery getSubQuery(SubQueryId subQueryId) {
+      return query.getSubQuery(subQueryId);
+    }
+
+    public QueryId getQueryId() {
+      return queryId;
+    }
+
+    public ApplicationId getApplicationId() {
+      return appId;
+    }
+
+    public ApplicationAttemptId getApplicationAttemptId() {
+      return appAttemptId;
+    }
+
+    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 long getStartTime() {
+      if(query != null) {
+        return query.getStartTime();
+      } else {
+        return -1;
+      }
+    }
+
+    public long getFinishTime() {
+      if(query != null) {
+        return query.getFinishTime();
+      } else {
+        return -1;
+      }
+    }
+
+    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> {
+    @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);
+      }
+
+      //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();
+
+        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
+      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());
+
+      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));
+      }
+    }
+
+    conf.setOutputPath(stagingDir);
+    outputPath = stagingDir;
+    LOG.info("Initialized Query Staging Dir: " + outputPath);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..1a326fe
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
@@ -0,0 +1,196 @@
+/**
+ * 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.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 = rpcServer.getBindAddress();
+      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+    }
+    // Get the master address
+    LOG.info(QueryMasterClientService.class.getSimpleName() + " is bind to " + addr);
+    //queryConf.setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, 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/9d020883/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
new file mode 100644
index 0000000..47adf7d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
@@ -0,0 +1,353 @@
+/**
+ * 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(1000);
+      } 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());
+
+    // 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(2048);
+    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/9d020883/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
new file mode 100644
index 0000000..a3c7b75
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
@@ -0,0 +1,114 @@
+/**
+ * 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.net.NetUtils;
+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 java.net.InetSocketAddress;
+
+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(TajoConf.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 = server.getBindAddress();
+    this.conf.setVar(TajoConf.ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS,
+            org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
+    LOG.info("Instantiated QueryMasterManagerService 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 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/9d020883/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
new file mode 100644
index 0000000..f34464b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
@@ -0,0 +1,152 @@
+/**
+ * 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.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.util.TajoIdUtils;
+
+import java.io.PrintWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+
+public class QueryMasterRunner extends AbstractService {
+  private static final Log LOG = LogFactory.getLog(QueryMasterRunner.class);
+  private QueryConf queryConf;
+  private QueryMaster queryMaster;
+  private QueryId queryId;
+  private long appSubmitTime;
+  private String queryMasterManagerAddress;
+
+  public QueryMasterRunner(QueryId queryId, long appSubmitTime, String queryMasterManagerAddress) {
+    super(QueryMasterRunner.class.getName());
+    this.queryId = queryId;
+    this.appSubmitTime = appSubmitTime;
+    this.queryMasterManagerAddress = queryMasterManagerAddress;
+  }
+
+  private class ShutdownHook implements Runnable {
+    @Override
+    public void run() {
+      LOG.info("============================================");
+      LOG.info("QueryMaster received SIGINT Signal");
+      LOG.info("============================================");
+      stop();
+    }
+  }
+
+  @Override
+  public void init(Configuration conf) {
+    this.queryConf = (QueryConf)conf;
+    RackResolver.init(queryConf);
+
+    Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    //create QueryMaster
+    QueryMaster query = new QueryMaster(queryId, appSubmitTime, queryMasterManagerAddress);
+
+    query.init(queryConf);
+    query.start();
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  public static void main(String[] args) throws Exception {
+    LOG.info("QueryMasterRunner started");
+
+    final QueryConf conf = new QueryConf();
+    conf.addResource(new Path(QueryConf.FILENAME));
+
+    UserGroupInformation.setConfiguration(conf);
+
+    final QueryId queryId = TajoIdUtils.createQueryId(args[0]);
+    final long appSubmitTime = Long.parseLong(args[1]);
+    final String queryMasterManagerAddr = args[2];
+
+    LOG.info("Received QueryId:" + queryId);
+
+    QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, appSubmitTime, queryMasterManagerAddr);
+    queryMasterRunner.init(conf);
+    queryMasterRunner.start();
+
+    synchronized(queryId) {
+      queryId.wait();
+    }
+
+    System.exit(0);
+  }
+
+  public static void printThreadInfo(PrintWriter stream, String title) {
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    final int STACK_DEPTH = 60;
+    boolean contention = threadBean.isThreadContentionMonitoringEnabled();
+    long[] threadIds = threadBean.getAllThreadIds();
+    stream.println("Process Thread Dump: " + title);
+    stream.println(threadIds.length + " active threads");
+    for (long tid : threadIds) {
+      ThreadInfo info = threadBean.getThreadInfo(tid, STACK_DEPTH);
+      if (info == null) {
+        stream.println("  Inactive");
+        continue;
+      }
+      stream.println("Thread " + getTaskName(info.getThreadId(), info.getThreadName()) + ":");
+      Thread.State state = info.getThreadState();
+      stream.println("  State: " + state);
+      stream.println("  Blocked count: " + info.getBlockedCount());
+      stream.println("  Waited count: " + info.getWaitedCount());
+      if (contention) {
+        stream.println("  Blocked time: " + info.getBlockedTime());
+        stream.println("  Waited time: " + info.getWaitedTime());
+      }
+      if (state == Thread.State.WAITING) {
+        stream.println("  Waiting on " + info.getLockName());
+      } else if (state == Thread.State.BLOCKED) {
+        stream.println("  Blocked on " + info.getLockName());
+        stream.println("  Blocked by " + getTaskName(info.getLockOwnerId(), info.getLockOwnerName()));
+      }
+      stream.println("  Stack:");
+      for (StackTraceElement frame : info.getStackTrace()) {
+        stream.println("    " + frame.toString());
+      }
+    }
+    stream.flush();
+  }
+
+  private static String getTaskName(long id, String name) {
+    if (name == null) {
+      return Long.toString(id);
+    }
+    return id + " (" + name + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..fcb8f3e
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
@@ -0,0 +1,500 @@
+/**
+ * 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.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.QueryIdFactory;
+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.master.TaskState;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.storage.Fragment;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class QueryUnit implements EventHandler<TaskEvent> {
+  /** Class Logger */
+  private static final Log LOG = LogFactory.getLog(QueryUnit.class);
+
+	private QueryUnitId taskId;
+  private EventHandler eventHandler;
+	private StoreTableNode store = null;
+	private LogicalNode plan = null;
+	private List<ScanNode> scan;
+	
+	private Map<String, Fragment> fragMap;
+	private Map<String, Set<URI>> fetchMap;
+	
+  private List<Partition> partitions;
+	private TableStat stats;
+  private String [] dataLocations;
+  private final boolean isLeafTask;
+  private List<IntermediateEntry> intermediateData;
+
+  private Map<QueryUnitAttemptId, QueryUnitAttempt> attempts;
+  private final int maxAttempts = 3;
+  private Integer lastAttemptId;
+
+  private QueryUnitAttemptId successfulAttempt;
+  private String succeededHost;
+  private int succeededPullServerPort;
+
+  private int failedAttempts;
+  private int finishedAttempts; // finish are total of success, failed and killed
+
+  private static final StateMachineFactory
+      <QueryUnit, TaskState, TaskEventType, TaskEvent> stateMachineFactory =
+      new StateMachineFactory
+          <QueryUnit, TaskState, TaskEventType, TaskEvent>(TaskState.NEW)
+
+      .addTransition(TaskState.NEW, TaskState.SCHEDULED,
+          TaskEventType.T_SCHEDULE, new InitialScheduleTransition())
+
+       .addTransition(TaskState.SCHEDULED, TaskState.RUNNING,
+           TaskEventType.T_ATTEMPT_LAUNCHED)
+
+        .addTransition(TaskState.RUNNING, TaskState.RUNNING,
+           TaskEventType.T_ATTEMPT_LAUNCHED)
+
+       .addTransition(TaskState.RUNNING, TaskState.SUCCEEDED,
+           TaskEventType.T_ATTEMPT_SUCCEEDED, new AttemptSucceededTransition())
+
+       .addTransition(TaskState.RUNNING,
+            EnumSet.of(TaskState.RUNNING, TaskState.FAILED),
+            TaskEventType.T_ATTEMPT_FAILED, new AttemptFailedTransition())
+
+
+
+      .installTopology();
+  private final StateMachine<TaskState, TaskEventType, TaskEvent> stateMachine;
+
+
+  private final Lock readLock;
+  private final Lock writeLock;
+
+	public QueryUnit(QueryUnitId id, boolean isLeafTask, EventHandler eventHandler) {
+		this.taskId = id;
+    this.eventHandler = eventHandler;
+    this.isLeafTask = isLeafTask;
+		scan = new ArrayList<ScanNode>();
+    fetchMap = Maps.newHashMap();
+    fragMap = Maps.newHashMap();
+    partitions = new ArrayList<Partition>();
+    attempts = Collections.emptyMap();
+    lastAttemptId = -1;
+    failedAttempts = 0;
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+
+    stateMachine = stateMachineFactory.make(this);
+	}
+
+  public boolean isLeafTask() {
+    return this.isLeafTask;
+  }
+
+  public void setDataLocations(String [] dataLocations) {
+    this.dataLocations = dataLocations;
+  }
+
+  public String [] getDataLocations() {
+    return this.dataLocations;
+  }
+
+  public TaskState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+	
+	public void setLogicalPlan(LogicalNode plan) {
+    Preconditions.checkArgument(plan.getType() == ExprType.STORE ||
+        plan.getType() == ExprType.CREATE_INDEX);
+    
+	  this.plan = plan;
+    store = (StoreTableNode) plan;
+
+	  LogicalNode node = plan;
+	  ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
+	  s.add(node);
+	  while (!s.isEmpty()) {
+	    node = s.remove(s.size()-1);
+	    if (node instanceof UnaryNode) {
+	      UnaryNode unary = (UnaryNode) node;
+	      s.add(s.size(), unary.getSubNode());
+	    } else if (node instanceof BinaryNode) {
+	      BinaryNode binary = (BinaryNode) node;
+	      s.add(s.size(), binary.getOuterNode());
+	      s.add(s.size(), binary.getInnerNode());
+	    } else if (node instanceof ScanNode) {
+	      scan.add((ScanNode)node);
+	    }
+	  }
+	}
+
+  @Deprecated
+  public void setFragment(String tableId, Fragment fragment) {
+    this.fragMap.put(tableId, fragment);
+    if (fragment.hasDataLocations()) {
+      setDataLocations(fragment.getDataLocations());
+    }
+  }
+
+  public void setFragment2(Fragment fragment) {
+    this.fragMap.put(fragment.getId(), fragment);
+    if (fragment.hasDataLocations()) {
+      setDataLocations(fragment.getDataLocations());
+    }
+  }
+	
+	public void addFetch(String tableId, String uri) throws URISyntaxException {
+	  this.addFetch(tableId, new URI(uri));
+	}
+	
+	public void addFetch(String tableId, URI uri) {
+	  Set<URI> uris;
+	  if (fetchMap.containsKey(tableId)) {
+	    uris = fetchMap.get(tableId);
+	  } else {
+	    uris = Sets.newHashSet();
+	  }
+	  uris.add(uri);
+    fetchMap.put(tableId, uris);
+	}
+	
+	public void addFetches(String tableId, Collection<URI> urilist) {
+	  Set<URI> uris;
+    if (fetchMap.containsKey(tableId)) {
+      uris = fetchMap.get(tableId);
+    } else {
+      uris = Sets.newHashSet();
+    }
+    uris.addAll(urilist);
+    fetchMap.put(tableId, uris);
+	}
+	
+	public void setFetches(Map<String, Set<URI>> fetches) {
+	  this.fetchMap.clear();
+	  this.fetchMap.putAll(fetches);
+	}
+	
+  public Fragment getFragment(String tableId) {
+    return this.fragMap.get(tableId);
+  }
+
+  public Collection<Fragment> getAllFragments() {
+    return fragMap.values();
+  }
+	
+	public LogicalNode getLogicalPlan() {
+	  return this.plan;
+	}
+	
+	public QueryUnitId getId() {
+		return taskId;
+	}
+	
+	public Collection<URI> getFetchHosts(String tableId) {
+	  return fetchMap.get(tableId);
+	}
+	
+	public Collection<Set<URI>> getFetches() {
+	  return fetchMap.values();
+	}
+	
+	public Collection<URI> getFetch(ScanNode scan) {
+	  return this.fetchMap.get(scan.getTableId());
+	}
+
+	public String getOutputName() {
+		return this.store.getTableName();
+	}
+	
+	public Schema getOutputSchema() {
+	  return this.store.getOutSchema();
+	}
+	
+	public StoreTableNode getStoreTableNode() {
+	  return this.store;
+	}
+	
+	public ScanNode[] getScanNodes() {
+	  return this.scan.toArray(new ScanNode[scan.size()]);
+	}
+	
+	@Override
+	public String toString() {
+		String str = new String(plan.getType() + " \n");
+		for (Entry<String, Fragment> e : fragMap.entrySet()) {
+		  str += e.getKey() + " : ";
+      str += e.getValue() + " ";
+		}
+		for (Entry<String, Set<URI>> e : fetchMap.entrySet()) {
+      str += e.getKey() + " : ";
+      for (URI t : e.getValue()) {
+        str += t + " ";
+      }
+    }
+		
+		return str;
+	}
+	
+	public void setStats(TableStat stats) {
+	  this.stats = stats;
+	}
+	
+	public void setPartitions(List<Partition> partitions) {
+	  this.partitions = Collections.unmodifiableList(partitions);
+	}
+	
+	public TableStat getStats() {
+	  return this.stats;
+	}
+	
+	public List<Partition> getPartitions() {
+	  return this.partitions;
+	}
+	
+	public int getPartitionNum() {
+	  return this.partitions.size();
+	}
+
+  public QueryUnitAttempt newAttempt() {
+    QueryUnitAttempt attempt = new QueryUnitAttempt(
+        QueryIdFactory.newQueryUnitAttemptId(this.getId(),
+            ++lastAttemptId), this, eventHandler);
+    return attempt;
+  }
+
+  public QueryUnitAttempt getAttempt(QueryUnitAttemptId attemptId) {
+    return attempts.get(attemptId);
+  }
+
+  public QueryUnitAttempt getAttempt(int attempt) {
+    return this.attempts.get(new QueryUnitAttemptId(this.getId(), attempt));
+  }
+
+  public QueryUnitAttempt getLastAttempt() {
+    return this.attempts.get(this.lastAttemptId);
+  }
+
+  protected QueryUnitAttempt getSuccessfulAttempt() {
+    readLock.lock();
+    try {
+      if (null == successfulAttempt) {
+        return null;
+      }
+      return attempts.get(successfulAttempt);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public int getRetryCount () {
+    return this.lastAttemptId;
+  }
+
+  private static class InitialScheduleTransition implements
+    SingleArcTransition<QueryUnit, TaskEvent> {
+
+    @Override
+    public void transition(QueryUnit task, TaskEvent taskEvent) {
+      task.addAndScheduleAttempt();
+    }
+  }
+
+  // This is always called in the Write Lock
+  private void addAndScheduleAttempt() {
+    // Create new task attempt
+    QueryUnitAttempt attempt = newAttempt();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Created attempt " + attempt.getId());
+    }
+    switch (attempts.size()) {
+      case 0:
+        attempts = Collections.singletonMap(attempt.getId(), attempt);
+        break;
+
+      case 1:
+        Map<QueryUnitAttemptId, QueryUnitAttempt> newAttempts
+            = new LinkedHashMap<QueryUnitAttemptId, QueryUnitAttempt>(3);
+        newAttempts.putAll(attempts);
+        attempts = newAttempts;
+        attempts.put(attempt.getId(), attempt);
+        break;
+
+      default:
+        attempts.put(attempt.getId(), attempt);
+        break;
+    }
+
+    if (failedAttempts > 0) {
+      eventHandler.handle(new TaskAttemptEvent(attempt.getId(),
+          TaskAttemptEventType.TA_RESCHEDULE));
+    } else {
+      eventHandler.handle(new TaskAttemptEvent(attempt.getId(),
+          TaskAttemptEventType.TA_SCHEDULE));
+    }
+  }
+
+  private static class AttemptSucceededTransition
+      implements SingleArcTransition<QueryUnit, TaskEvent>{
+
+    @Override
+    public void transition(QueryUnit task,
+                           TaskEvent event) {
+      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+      QueryUnitAttempt attempt = task.attempts.get(
+          attemptEvent.getTaskAttemptId());
+      task.successfulAttempt = attemptEvent.getTaskAttemptId();
+      task.succeededHost = attempt.getHost();
+      task.succeededPullServerPort = attempt.getPullServerPort();
+      task.eventHandler.handle(new SubQueryTaskEvent(event.getTaskId(),
+          SubQueryEventType.SQ_TASK_COMPLETED));
+    }
+  }
+
+  private static class AttemptFailedTransition implements
+    MultipleArcTransition<QueryUnit, TaskEvent, TaskState> {
+
+    @Override
+    public TaskState transition(QueryUnit task, TaskEvent taskEvent) {
+      TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) taskEvent;
+      LOG.info("=============================================================");
+      LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + " <<<");
+      LOG.info("=============================================================");
+      task.failedAttempts++;
+      task.finishedAttempts++;
+
+      if (task.failedAttempts < task.maxAttempts) {
+        if (task.successfulAttempt == null) {
+          task.addAndScheduleAttempt();
+        }
+      } else {
+        task.eventHandler.handle(
+            new SubQueryTaskEvent(task.getId(), SubQueryEventType.SQ_FAILED));
+        return TaskState.FAILED;
+      }
+
+      return task.getState();
+    }
+  }
+
+  @Override
+  public void handle(TaskEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getTaskId() + " of type "
+          + event.getType());
+    }
+
+    try {
+      writeLock.lock();
+      TaskState oldState = getState();
+      try {
+        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(),
+            QueryEventType.INTERNAL_ERROR));
+      }
+
+      //notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+        if (oldState != getState()) {
+          LOG.debug(taskId + " Task Transitioned from " + oldState + " to "
+              + getState());
+        }
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  public void setIntermediateData(Collection<IntermediateEntry> partitions) {
+    this.intermediateData = new ArrayList<IntermediateEntry>(partitions);
+  }
+
+  public List<IntermediateEntry> getIntermediateData() {
+    return this.intermediateData;
+  }
+
+  public static class IntermediateEntry {
+    int taskId;
+    int attemptId;
+    int partitionId;
+    String pullHost;
+    int port;
+
+    public IntermediateEntry(int taskId, int attemptId, int partitionId,
+                             String pullServerAddr, int pullServerPort) {
+      this.taskId = taskId;
+      this.attemptId = attemptId;
+      this.partitionId = partitionId;
+      this.pullHost = pullServerAddr;
+      this.port = pullServerPort;
+    }
+
+    public int getTaskId() {
+      return this.taskId;
+    }
+
+    public int getAttemptId() {
+      return this.attemptId;
+    }
+
+    public int getPartitionId() {
+      return this.partitionId;
+    }
+
+    public String getPullHost() {
+      return this.pullHost;
+    }
+
+    public int getPullPort() {
+      return port;
+    }
+
+    public String getPullAddress() {
+      return pullHost + ":" + port;
+    }
+  }
+}


[2/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..2c5c2b6
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
@@ -0,0 +1,136 @@
+/**
+ * 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 = "ClientProtos";
+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";
+
+enum ResultCode {
+  OK = 0;
+  ERROR = 1;
+}
+
+message UpdateSessionVariableRequest {
+  optional SessionIdProto sessionId = 1;
+  repeated KeyValueProto setVariables = 2;
+  repeated string unsetVariables = 3;
+}
+
+message QueryRequest {
+  optional SessionIdProto sessionId = 1;
+  required string query = 2;
+  repeated KeyValueProto setVariables = 3;
+}
+
+message UpdateQueryResponse {
+  required ResultCode resultCode = 1;
+  optional string errorMessage = 2;
+}
+
+message SubmitQueryResponse {
+  required ResultCode resultCode = 1;
+  optional ApplicationAttemptIdProto queryId = 2;
+  optional string errorMessage = 3;
+}
+
+message GetQueryResultRequest {
+  optional SessionIdProto sessionId = 1;
+  required ApplicationAttemptIdProto queryId = 2;
+}
+
+message GetQueryResultResponse {
+  optional TableDescProto tableDesc = 1;
+  optional string errorMessage = 2;
+}
+
+message GetQueryListRequest {
+  optional SessionIdProto sessionId = 1;
+}
+
+message BriefQueryStatus {
+  required ApplicationAttemptIdProto queryId = 1;
+  required QueryState state = 2;
+  required int32 executionTime = 3;
+}
+
+message GetQueryListResponse {
+  repeated BriefQueryStatus queryList = 1;
+}
+
+message GetQueryStatusRequest {
+  optional SessionIdProto sessionId = 1;
+  required ApplicationAttemptIdProto queryId = 2;
+}
+
+message GetQueryStatusResponse {
+  required ResultCode resultCode = 1;
+  required ApplicationAttemptIdProto queryId = 2;
+  optional QueryState state = 3;
+  optional float progress = 4;
+  optional int64 submitTime = 5;
+  optional int64 initTime = 6;
+  optional int64 finishTime = 7;
+  optional bool hasResult = 8;
+  optional string errorMessage = 9;
+  optional string queryMasterHost = 10;
+  optional int32 queryMasterPort = 11;
+}
+
+message GetClusterInfoRequest {
+  optional SessionIdProto sessionId = 1;
+}
+
+message GetClusterInfoResponse {
+  repeated string serverName = 1;
+}
+
+message GetTableListRequest {
+  optional SessionIdProto sessionId = 1;
+}
+
+message GetTableListResponse {
+  repeated string tables = 1;
+}
+
+message GetTableDescRequest {
+  optional SessionIdProto sessionId = 1;
+  required string tableName = 2;
+}
+
+message CreateTableRequest {
+  required string name = 1;
+  required string path = 2;
+  required TableProto meta = 3;
+}
+
+message AttachTableRequest {
+  required string name = 1;
+  required string path = 2;
+}
+
+message TableResponse {
+  optional TableDescProto tableDesc = 1;
+  optional string errorMessage = 2;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtocol.proto
deleted file mode 100644
index 1ddf24a..0000000
--- a/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtocol.proto
+++ /dev/null
@@ -1,36 +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 = "MasterWorkerProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "MasterWorkerProtos.proto";
-import "PrimitiveProtos.proto";
-import "TajoIdProtos.proto";
-import "tajo_protos.proto";
-import "yarn_protos.proto";
-
-service MasterWorkerProtocolService {
-  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);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtos.proto
deleted file mode 100644
index ffffdcc..0000000
--- a/tajo-core/tajo-core-backend/src/main/proto/MasterWorkerProtos.proto
+++ /dev/null
@@ -1,118 +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.engine";
-option java_outer_classname = "MasterWorkerProtos";
-option java_generic_services = false;
-option java_generate_equals_and_hash = true;
-
-import "CatalogProtos.proto";
-import "TajoIdProtos.proto";
-import "tajo_protos.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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..9337078
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
@@ -0,0 +1,36 @@
+/**
+ * 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 = "QueryMasterClientProtocol";
+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";
+import "ClientProtos.proto";
+
+service QueryMasterClientProtocolService {
+  rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
+  rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
+  rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
+  rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..08fc5c9
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
@@ -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.
+ */
+
+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/9d020883/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
new file mode 100644
index 0000000..b6a0602
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
@@ -0,0 +1,132 @@
+/**
+ * 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/9d020883/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
new file mode 100644
index 0000000..ef7e711
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
@@ -0,0 +1,64 @@
+/**
+ * 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 = "TajoMasterClientProtocol";
+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";
+import "ClientProtos.proto";
+
+service TajoMasterClientProtocolService {
+  rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
+  rpc submitQuery(QueryRequest) returns (SubmitQueryResponse);
+  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 getClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse);
+  rpc existTable(StringProto) returns (BoolProto);
+  rpc getTableList(GetTableListRequest) returns (GetTableListResponse);
+  rpc getTableDesc(GetTableDescRequest) returns (TableResponse);
+  rpc createTable(CreateTableRequest) returns (TableResponse);
+  rpc dropTable(StringProto) returns (BoolProto);
+  rpc attachTable(AttachTableRequest) returns (TableResponse);
+  rpc detachTable(StringProto) returns (BoolProto);
+
+
+  // TODO - to be implemented
+  //
+  // authenticate
+  //
+  // getSessionVariableList
+  // dropTable
+  // detachTable
+  // createIndex
+  // dropIndex
+  // registerUDF
+  // dropUDF
+  // listUdfs
+  // getUDFDesc
+  // registerJars
+  // getListRegisteredJars
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/proto/tajo_protos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/tajo_protos.proto b/tajo-core/tajo-core-backend/src/main/proto/tajo_protos.proto
index 3d22ff5..d337315 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/tajo_protos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/tajo_protos.proto
@@ -22,13 +22,16 @@ option java_generic_services = false;
 option java_generate_equals_and_hash = true;
 
 enum QueryState {
-  QUERY_NEW = 0;
-  QUERY_INIT = 1;
-  QUERY_RUNNING = 2;
-  QUERY_SUCCEEDED = 3;
-  QUERY_FAILED = 4;
-  QUERY_KILLED = 5;
-  QUERY_ERROR = 6;
+  QUERY_MASTER_INIT = 0;
+  QUERY_MASTER_LAUNCHED = 1;
+  QUERY_NEW = 2;
+  QUERY_INIT = 3;
+  QUERY_RUNNING = 4;
+  QUERY_SUCCEEDED = 5;
+  QUERY_FAILED = 6;
+  QUERY_KILLED = 7;
+  QUERY_ERROR = 8;
+  QUERY_NOT_ASSIGNED = 9;
 }
 
 enum TaskAttemptState {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 c003637..f3da922 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
@@ -35,4 +35,22 @@
     <name>tajo.task.localdir</name>
     <value>/tmp/tajo-localdir</value>
   </property>
+
+  <property>
+    <name>tajo.master.clientservice.addr</name>
+    <value>127.0.0.1:9004</value>
+  </property>
+
+  <property>
+    <name>tajo.master.querymastermanager.addr</name>
+    <value>127.0.0.1:9005</value>
+  </property>
+
+  <property>
+    <name>tajo.query.session.timeout</name>
+    <value>60000</value>
+    <description>ms</description>
+  </property>
+
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/BackendTestingUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/BackendTestingUtil.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/BackendTestingUtil.java
index b22e893..e629623 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/BackendTestingUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/BackendTestingUtil.java
@@ -129,7 +129,7 @@ public class BackendTestingUtil {
     PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf, sm);
     PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
 
-    return new ResultSetImpl(conf, new Path(workDir, "out"));
+    return new ResultSetImpl(null, null, conf, new Path(workDir, "out"));
   }
 
   public static Path createTmpTestDir() throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 6425192..5e8d11d 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
@@ -19,6 +19,8 @@
 package org.apache.tajo;
 
 import com.google.protobuf.ServiceException;
+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.*;
@@ -30,6 +32,8 @@ import java.io.IOException;
 import java.sql.ResultSet;
 
 public class LocalTajoTestingUtility {
+  private static final Log LOG = LogFactory.getLog(LocalTajoTestingUtility.class);
+
   private TajoTestingCluster util;
   private TajoConf conf;
   private TajoClient client;
@@ -38,6 +42,9 @@ public class LocalTajoTestingUtility {
                     String[] tablepaths,
                     Schema[] schemas,
                     Options option) throws Exception {
+    LOG.info("===================================================");
+    LOG.info("Starting Test Cluster.");
+    LOG.info("===================================================");
 
     util = new TajoTestingCluster();
     util.startMiniCluster(1);
@@ -60,6 +67,11 @@ public class LocalTajoTestingUtility {
           CatalogProtos.StoreType.CSV, option);
       client.createTable(names[i], tablePath, meta);
     }
+
+    LOG.info("===================================================");
+    LOG.info("Test Cluster ready and test table created.");
+    LOG.info("===================================================");
+
   }
 
   public TajoTestingCluster getTestingCluster() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 8de0f0b..5b7267f 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
@@ -108,6 +108,11 @@ public class MiniTajoYarnCluster extends MiniYARNCluster {
     // for corresponding uberized tests.
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
 
+    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/9d020883/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 4911b48..fab2727 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,6 +27,7 @@ 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.tajo.catalog.*;
@@ -154,7 +155,7 @@ public class TajoTestingCluster {
     System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
         this.clusterTestBuildDir.toString());
 
-    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(new HdfsConfiguration(conf));
     builder.hosts(hosts);
     builder.numDataNodes(servers);
     builder.format(true);
@@ -230,6 +231,8 @@ 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.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");
@@ -392,13 +395,19 @@ public class TajoTestingCluster {
   }
 
   public void shutdownMiniCluster() throws IOException {
-    LOG.info("Shutting down minicluster");
+    LOG.info("========================================");
+    LOG.info("Shutdown minicluster");
+    LOG.info("========================================");
     shutdownMiniTajoCluster();
 
     if(this.catalogServer != null) {
       shutdownCatalogCluster();
     }
 
+    if(this.yarnCluster != null) {
+      this.yarnCluster.stop();
+    }
+
     if(this.dfsCluster != null) {
       this.dfsCluster.shutdown();
     }
@@ -442,38 +451,6 @@ public class TajoTestingCluster {
   }
 
   public static ResultSet run(String[] names,
-                              String[] tablepaths,
-                              Schema[] schemas,
-                              Options option,
-                              String query) throws Exception {
-    TajoTestingCluster util = new TajoTestingCluster();
-    util.startMiniCluster(1);
-    TajoConf conf = util.getConfiguration();
-    TajoClient client = new TajoClient(conf);
-
-    FileSystem fs = util.getDefaultFileSystem();
-    Path rootDir = util.getMaster().
-        getStorageManager().getBaseDir();
-    fs.mkdirs(rootDir);
-    for (int i = 0; i < tablepaths.length; i++) {
-      Path localPath = new Path(tablepaths[i]);
-      Path tablePath = new Path(rootDir, names[i]);
-      fs.mkdirs(tablePath);
-      Path dataPath = new Path(tablePath, "data");
-      fs.mkdirs(dataPath);
-      Path dfsPath = new Path(dataPath, localPath.getName());
-      fs.copyFromLocalFile(localPath, dfsPath);
-      TableMeta meta = CatalogUtil.newTableMeta(schemas[i],
-          CatalogProtos.StoreType.CSV, option);
-      client.createTable(names[i], tablePath, meta);
-    }
-    Thread.sleep(1000);
-    ResultSet res = client.executeQueryAndGetResult(query);
-    util.shutdownMiniCluster();
-    return res;
-  }
-
-  public static ResultSet run(String[] names,
                               Schema[] schemas,
                               Options option,
                               String[][] tables,
@@ -554,23 +531,5 @@ public class TajoTestingCluster {
     TajoTestingCluster cluster2 = new TajoTestingCluster();
     File f2 = cluster2.setupClusterTestBuildDir();
     System.out.println("first setupClusterTestBuildDir of cluster2: " + f2);
-    /*
-    String [] names = {"table1"};
-    String [][] tables = new String[1][];
-    tables[0] = new String[] {"a,b,c", "b,c,d"};
-
-    Schema [] schemas = new Schema[1];
-    schemas[0] = new Schema()
-          .addColumn("f1", CatalogProtos.DataType.STRING)
-          .addColumn("f2", CatalogProtos.DataType.STRING)
-          .addColumn("f3", CatalogProtos.DataType.STRING);
-
-    ResultSet res = runInLocal(names, schemas, tables, "select f1 from table1");
-    res.next();
-    System.out.println(res.getString(0));
-    res.next();
-    System.out.println(res.getString(0));
-    System.exit(0);
-    */
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 bc3b91d..c761103 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
@@ -51,7 +51,7 @@ public class TpchTestBase {
       testBase.setUp();
       Runtime.getRuntime().addShutdownHook(new ShutdownHook());
     } catch (Exception e) {
-      LOG.error(e);
+      LOG.error(e.getMessage(), e);
     }
   }
 
@@ -75,6 +75,9 @@ public class TpchTestBase {
     File file;
     for (int i = 0; i < names.length; i++) {
       file = new File("src/test/tpch/" + names[i] + ".tbl");
+      if(!file.exists()) {
+        file = new File(System.getProperty("user.dir") + "/tajo-core/tajo-core-backend/src/test/tpch/" + names[i] + ".tbl");
+      }
       tables[i] = FileUtil.readTextFile(file).split("\n");
       paths[i] = file.getAbsolutePath();
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/benchmark/TestTPCH.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
index b47c635..b61dc46 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/benchmark/TestTPCH.java
@@ -53,15 +53,20 @@ public class TestTPCH {
     ResultSet res = tpch.execute("select l_returnflag, l_linestatus, count(*) as count_order from lineitem " +
         "group by l_returnflag, l_linestatus order by l_returnflag, l_linestatus");
 
-    Map<String,Integer> result = Maps.newHashMap();
-    result.put("NO", 3);
-    result.put("RF", 2);
+    try {
+      Map<String,Integer> result = Maps.newHashMap();
+      result.put("NO", 3);
+      result.put("RF", 2);
 
-    res.next();
-    assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-    res.next();
-    assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
-    assertFalse(res.next());
+      assertNotNull(res);
+      assertTrue(res.next());
+      assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
+      assertTrue(res.next());
+      assertTrue(result.get(res.getString(1) + res.getString(2)) == res.getInt(3));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -74,12 +79,16 @@ public class TestTPCH {
             "join partsupp on s_suppkey = ps_suppkey " +
             "join part on p_partkey = ps_partkey and p_type like '%BRASS' and p_size = 15");
 
-    assertTrue(res.next());
-    assertEquals("AMERICA", res.getString(10));
-    String [] pType = res.getString(11).split(" ");
-    assertEquals("BRASS", pType[pType.length - 1]);
-    assertEquals(15, res.getInt(12));
-    assertFalse(res.next());
+    try {
+      assertTrue(res.next());
+      assertEquals("AMERICA", res.getString(10));
+      String [] pType = res.getString(11).split(" ");
+      assertEquals("BRASS", pType[pType.length - 1]);
+      assertEquals(15, res.getInt(12));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -88,7 +97,11 @@ public class TestTPCH {
         "case when p_type like 'PROMO%' then l_extendedprice else 0.0 end) / sum(l_extendedprice * (1 - l_discount)) "
         + "as promo_revenue from lineitem, part where l_partkey = p_partkey");
 
-    res.next();
-    assertEquals(33, res.getInt(1));
+    try {
+      assertTrue(res.next());
+      assertEquals(33, res.getInt(1));
+    } finally {
+      res.close();
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
index 67abc95..9f50fe3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/client/TestTajoClient.java
@@ -62,7 +62,9 @@ public class TestTajoClient {
   @AfterClass
   public static void tearDown() throws Exception {
     util.shutdownMiniCluster();
-    tajo.close();
+    if(tajo != null) {
+      tajo.close();
+    }
   }
 
   private static Path writeTmpTable(String tableName) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
index 79c21ee..6f09fa6 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
@@ -49,22 +49,34 @@ public class TestBuiltinFunctions {
   @Test
   public void testMaxLong() throws Exception {
     ResultSet res = tpch.execute("select max(l_orderkey) as max from lineitem");
-    res.next();
-    assertEquals(3, res.getInt(1));
+    try {
+      res.next();
+      assertEquals(3, res.getInt(1));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public void testMinLong() throws Exception {
     ResultSet res = tpch.execute("select min(l_orderkey) as max from lineitem");
-    res.next();
-    assertEquals(1, res.getInt(1));
+    try {
+      res.next();
+      assertEquals(1, res.getInt(1));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public void testCount() throws Exception {
     ResultSet res = tpch.execute("select count(*) as rownum from lineitem");
-    res.next();
-    assertEquals(5, res.getInt(1));
+    try {
+      res.next();
+      assertEquals(5, res.getInt(1));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -76,32 +88,48 @@ public class TestBuiltinFunctions {
 
     ResultSet res = tpch.execute("select l_orderkey, avg(l_discount) as revenue from lineitem group by l_orderkey");
 
-    while(res.next()) {
-      assertTrue(result.get(res.getLong(1)) == res.getFloat(2));
+    try {
+      while(res.next()) {
+        assertTrue(result.get(res.getLong(1)) == res.getFloat(2));
+      }
+    } finally {
+      res.close();
     }
   }
 
   @Test
   public void testAvgLong() throws Exception {
     ResultSet res = tpch.execute("select avg(l_orderkey) as avg from lineitem");
-    res.next();
-    assertEquals(2, res.getLong(1));
+    try {
+      res.next();
+      assertEquals(2, res.getLong(1));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public void testAvgInt() throws Exception {
     ResultSet res = tpch.execute("select avg(l_partkey) as avg from lineitem");
-    res.next();
-    System.out.println(res.getFloat(1));
-    assertTrue(1.8f == res.getFloat(1));
+    try {
+      res.next();
+      System.out.println(res.getFloat(1));
+      assertTrue(1.8f == res.getFloat(1));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public void testRandom() throws Exception {
     ResultSet res = tpch.execute("select l_orderkey, random(3) as rndnum from lineitem group by l_orderkey, rndnum");
 
-    while(res.next()) {
-      assertTrue(res.getInt(2) >= 0 && res.getInt(2) < 3);
+    try {
+      while(res.next()) {
+        assertTrue(res.getInt(2) >= 0 && res.getInt(2) < 3);
+      }
+    } finally {
+      res.close();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 aa2f77c..cf0d70a 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
@@ -100,7 +100,7 @@ public class TestGlobalQueryPlanner {
     dispatcher.init(conf);
     dispatcher.start();
 
-    planner = new GlobalPlanner(conf, catalog, new StorageManager(conf),
+    planner = new GlobalPlanner(conf, new StorageManager(conf),
         dispatcher.getEventHandler());
     analyzer = new SQLAnalyzer();
     logicalPlanner = new LogicalPlanner(catalog);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 c2e232b..5c303c0 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
@@ -91,7 +91,7 @@ public class TestGlobalQueryOptimizer {
 
     AsyncDispatcher dispatcher = new AsyncDispatcher();
 
-    planner = new GlobalPlanner(conf, catalog, new StorageManager(conf),
+    planner = new GlobalPlanner(conf, new StorageManager(conf),
         dispatcher.getEventHandler());
     analyzer = new SQLAnalyzer();
     logicalPlanner = new LogicalPlanner(catalog);
@@ -132,7 +132,7 @@ public class TestGlobalQueryOptimizer {
     queryId = QueryIdFactory.newQueryId();
     optimizer = new GlobalOptimizer();
   }
-  
+
   @AfterClass
   public static void terminate() throws IOException {
     util.shutdownCatalogCluster();
@@ -147,7 +147,7 @@ public class TestGlobalQueryOptimizer {
 
     MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
     globalPlan = optimizer.optimize(globalPlan);
-    
+
     ExecutionBlock unit = globalPlan.getRoot();
     StoreTableNode store = unit.getStoreTableNode();
     assertEquals(ExprType.PROJECTION, store.getSubNode().getType());
@@ -156,14 +156,14 @@ public class TestGlobalQueryOptimizer {
     SortNode sort = (SortNode) proj.getSubNode();
     assertEquals(ExprType.SCAN, sort.getSubNode().getType());
     ScanNode scan = (ScanNode) sort.getSubNode();
-    
+
     assertTrue(unit.hasChildBlock());
     unit = unit.getChildBlock(scan);
     store = unit.getStoreTableNode();
     assertEquals(ExprType.SORT, store.getSubNode().getType());
     sort = (SortNode) store.getSubNode();
     assertEquals(ExprType.JOIN, sort.getSubNode().getType());
-    
+
     assertTrue(unit.hasChildBlock());
     for (ScanNode prevscan : unit.getScanNodes()) {
       ExecutionBlock prev = unit.getChildBlock(prevscan);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
index 61a9afe..cc497a8 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
@@ -50,37 +50,54 @@ public class TestGroupByQuery {
   public final void testComplexParameter() throws Exception {
     ResultSet res = tpch.execute(
         "select sum(l_extendedprice*l_discount) as revenue from lineitem");
-    assertTrue(res.next());
-    assertTrue(12908 == (int) res.getDouble("revenue"));
-    assertFalse(res.next());
+    try {
+      assertNotNull(res);
+      assertTrue(res.next());
+      assertTrue(12908 == (int) res.getDouble("revenue"));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testComplexParameter2() throws Exception {
     ResultSet res = tpch.execute("select count(*) + max(l_orderkey) as merged from lineitem");
-    res.next();
-    assertEquals(8, res.getLong("merged"));
+    try {
+      assertTrue(res.next());
+      assertEquals(8, res.getLong("merged"));
+    } finally {
+      res.close();
+    }
   }
 
   //@Test
   public final void testCube() throws Exception {
     ResultSet res = tpch.execute(
         "cube_test := select l_orderkey, l_partkey, sum(l_quantity) from lineitem group by cube(l_orderkey, l_partkey)");
-    int count = 0;
-    for (;res.next();) {
-      count++;
+    try {
+      int count = 0;
+      for (;res.next();) {
+        count++;
+      }
+      assertEquals(11, count);
+    } finally {
+      res.close();
     }
-    assertEquals(11, count);
   }
 
   //@Test
   // TODO - to fix the limit processing and then enable it
   public final void testGroupByLimit() throws Exception {
     ResultSet res = tpch.execute("select l_orderkey from lineitem limit 2");
-    int count = 0;
-    for (;res.next();) {
-      count++;
+    try {
+      int count = 0;
+      for (;res.next();) {
+        count++;
+      }
+      assertEquals(2, count);
+    } finally {
+      res.close();
     }
-    assertEquals(2, count);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index e11666c..233e9ee 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -56,24 +56,32 @@ public class TestJoinQuery {
   public final void testCrossJoin() throws Exception {
     ResultSet res = tpch.execute("select n_name, r_name, n_regionkey, r_regionkey from nation, region");
 
-    int cnt = 0;
-    while(res.next()) {
-      cnt++;
+    try {
+      int cnt = 0;
+      while(res.next()) {
+        cnt++;
+      }
+      // TODO - to check their joined contents
+      assertEquals(25 * 5, cnt);
+    } finally {
+      res.close();
     }
-    // TODO - to check their joined contents
-    assertEquals(25 * 5, cnt);
   }
 
   @Test
   public final void testCrossJoinWithExplicitJoinQual() throws Exception {
     ResultSet res = tpch.execute(
         "select n_name, r_name, n_regionkey, r_regionkey from nation, region where n_regionkey = r_regionkey");
-    int cnt = 0;
-    while(res.next()) {
-      cnt++;
+    try {
+      int cnt = 0;
+      while(res.next()) {
+        cnt++;
+      }
+      // TODO - to check their joined contents
+      assertEquals(25, cnt);
+    } finally {
+      res.close();
     }
-    // TODO - to check their joined contents
-    assertEquals(25, cnt);
   }
 
   @Test
@@ -81,49 +89,57 @@ public class TestJoinQuery {
     ResultSet res = tpch.execute(FileUtil
         .readTextFile(new File("src/test/queries/tpch_q2_simplified.tql")));
 
-    Object [][] result = new Object[3][3];
-
-    int tupleId = 0;
-    int colId = 0;
-    result[tupleId][colId++] = 4032.68f;
-    result[tupleId][colId++] = "Supplier#000000002";
-    result[tupleId++][colId] = "ETHIOPIA";
-
-    colId = 0;
-    result[tupleId][colId++] = 4641.08f;
-    result[tupleId][colId++] = "Supplier#000000004";
-    result[tupleId++][colId] = "MOROCCO";
-
-    colId = 0;
-    result[tupleId][colId++] = 4192.4f;
-    result[tupleId][colId++] = "Supplier#000000003";
-    result[tupleId][colId] = "ARGENTINA";
-
-    Map<Float, Object[]> resultSet =
-        Maps.newHashMap();
-    for (Object [] t : result) {
-      resultSet.put((Float) t[0], t);
+    try {
+      Object [][] result = new Object[3][3];
+
+      int tupleId = 0;
+      int colId = 0;
+      result[tupleId][colId++] = 4032.68f;
+      result[tupleId][colId++] = "Supplier#000000002";
+      result[tupleId++][colId] = "ETHIOPIA";
+
+      colId = 0;
+      result[tupleId][colId++] = 4641.08f;
+      result[tupleId][colId++] = "Supplier#000000004";
+      result[tupleId++][colId] = "MOROCCO";
+
+      colId = 0;
+      result[tupleId][colId++] = 4192.4f;
+      result[tupleId][colId++] = "Supplier#000000003";
+      result[tupleId][colId] = "ARGENTINA";
+
+      Map<Float, Object[]> resultSet =
+          Maps.newHashMap();
+      for (Object [] t : result) {
+        resultSet.put((Float) t[0], t);
+      }
+
+      for (int i = 0; i < 3; i++) {
+        res.next();
+        Object [] resultTuple = resultSet.get(res.getFloat("s_acctbal"));
+        assertEquals(resultTuple[0], res.getFloat("s_acctbal"));
+        assertEquals(resultTuple[1], res.getString("s_name"));
+        assertEquals(resultTuple[2], res.getString("n_name"));
+      }
+
+      assertFalse(res.next());
+    } finally {
+      res.close();
     }
-
-    for (int i = 0; i < 3; i++) {
-      res.next();
-      Object [] resultTuple = resultSet.get(res.getFloat("s_acctbal"));
-      assertEquals(resultTuple[0], res.getFloat("s_acctbal"));
-      assertEquals(resultTuple[1], res.getString("s_name"));
-      assertEquals(resultTuple[2], res.getString("n_name"));
-    }
-
-    assertFalse(res.next());
   }
 
   @Test
   public void testJoinRefEval() throws Exception {
-    ResultSet res = tpch.execute("select r_regionkey, n_regionkey, (r_regionkey + n_regionkey) as plus from region join nation on r_regionkey = n_regionkey");
-    int r, n;
-    while(res.next()) {
-      r = res.getInt(1);
-      n = res.getInt(2);
-      assertEquals(r + n, res.getInt(3));
+    ResultSet res = tpch.execute("select r_regionkey, n_regionkey, (r_regionkey + n_regionkey) as plus from region, nation where r_regionkey = n_regionkey");
+    try {
+      int r, n;
+      while(res.next()) {
+        r = res.getInt(1);
+        n = res.getInt(2);
+        assertEquals(r + n, res.getInt(3));
+      }
+    } finally {
+      res.close();
     }
   }
 
@@ -137,16 +153,18 @@ public class TestJoinQuery {
         "else 'zero' " +
         "end as cond from region, nation where r_regionkey = n_regionkey");
 
-
-
-    Map<Integer, String> result = Maps.newHashMap();
-    result.put(0, "zero");
-    result.put(1, "one");
-    result.put(2, "two");
-    result.put(3, "three");
-    result.put(4, "four");
-    while(res.next()) {
-      assertEquals(result.get(res.getInt(1)), res.getString(3));
+    try {
+      Map<Integer, String> result = Maps.newHashMap();
+      result.put(0, "zero");
+      result.put(1, "one");
+      result.put(2, "two");
+      result.put(3, "three");
+      result.put(4, "four");
+      while(res.next()) {
+        assertEquals(result.get(res.getInt(1)), res.getString(3));
+      }
+    } finally {
+      res.close();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestNullValues.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
index 74ec2b5..2d478dd 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestNullValues.java
@@ -55,9 +55,13 @@ public class TestNullValues {
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable1 where col2 is null");
-    assertTrue(res.next());
-    assertEquals(2, res.getInt(1));
-    assertFalse(res.next());
+    try {
+      assertTrue(res.next());
+      assertEquals(2, res.getInt(1));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -77,11 +81,15 @@ public class TestNullValues {
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable2 where col2 is not null");
-    assertTrue(res.next());
-    assertEquals(1, res.getInt(1));
-    assertTrue(res.next());
-    assertEquals(3, res.getInt(1));
-    assertFalse(res.next());
+    try {
+      assertTrue(res.next());
+      assertEquals(1, res.getInt(1));
+      assertTrue(res.next());
+      assertEquals(3, res.getInt(1));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -108,8 +116,12 @@ public class TestNullValues {
     ResultSet res = TajoTestingCluster
         .run(table, schemas, opts, new String[][]{data},
             "select * from nulltable3 where col1 is null and col2 is null and col3 is null and col4 = 43578");
-    assertTrue(res.next());
-    assertEquals(43578, res.getLong(4));
-    assertFalse(res.next());
+    try {
+      assertTrue(res.next());
+      assertEquals(43578, res.getLong(4));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
index 7a04a40..a16d0f3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
@@ -100,7 +100,7 @@ public class TestResultSetImpl {
 
   @Test
   public void test() throws IOException, SQLException {
-    ResultSetImpl rs = new ResultSetImpl(conf, sm.getTablePath("score"));
+    ResultSetImpl rs = new ResultSetImpl(null, null, conf, sm.getTablePath("score"));
     ResultSetMetaData meta = rs.getMetaData();
     assertNotNull(meta);
     Schema schema = scoreMeta.getSchema();

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index a47f5bd..2a189f8 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -48,90 +48,106 @@ public class TestSelectQuery {
   @Test
   public final void testSelect() throws Exception {
     ResultSet res = tpch.execute("select l_orderkey, l_partkey from lineitem");
-    res.next();
-    assertEquals(1, res.getInt(1));
-    assertEquals(1, res.getInt(2));
-
-    res.next();
-    assertEquals(1, res.getInt(1));
-    assertEquals(1, res.getInt(2));
-    assertEquals(1, res.getInt(2));
-
-    res.next();
-    assertEquals(2, res.getInt(1));
-    assertEquals(2, res.getInt(2));
+    try {
+      res.next();
+      assertEquals(1, res.getInt(1));
+      assertEquals(1, res.getInt(2));
+
+      res.next();
+      assertEquals(1, res.getInt(1));
+      assertEquals(1, res.getInt(2));
+      assertEquals(1, res.getInt(2));
+
+      res.next();
+      assertEquals(2, res.getInt(1));
+      assertEquals(2, res.getInt(2));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testSelect2() throws Exception {
     ResultSet res = tpch.execute("select l_orderkey, l_partkey, l_orderkey + l_partkey as plus from lineitem");
-    res.next();
-    assertEquals(1, res.getInt(1));
-    assertEquals(1, res.getInt(2));
-    assertEquals(2, res.getInt(3));
-
-    res.next();
-    assertEquals(1, res.getInt(1));
-    assertEquals(1, res.getInt(2));
-    assertEquals(2, res.getInt(3));
-
-    res.next();
-    assertEquals(2, res.getInt(1));
-    assertEquals(2, res.getInt(2));
-    assertEquals(4, res.getInt(3));
+    try {
+      res.next();
+      assertEquals(1, res.getInt(1));
+      assertEquals(1, res.getInt(2));
+      assertEquals(2, res.getInt(3));
+
+      res.next();
+      assertEquals(1, res.getInt(1));
+      assertEquals(1, res.getInt(2));
+      assertEquals(2, res.getInt(3));
+
+      res.next();
+      assertEquals(2, res.getInt(1));
+      assertEquals(2, res.getInt(2));
+      assertEquals(4, res.getInt(3));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testSelect3() throws Exception {
     ResultSet res = tpch.execute("select l_orderkey + l_partkey as plus from lineitem");
-    res.next();
-    assertEquals(2, res.getInt(1));
+    try {
+      res.next();
+      assertEquals(2, res.getInt(1));
 
-    res.next();
-    assertEquals(2, res.getInt(1));
+      res.next();
+      assertEquals(2, res.getInt(1));
 
-    res.next();
-    assertEquals(4, res.getInt(1));
+      res.next();
+      assertEquals(4, res.getInt(1));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testSelectAsterik() throws Exception {
     ResultSet res = tpch.execute("select * from lineitem");
-    res.next();
-    assertEquals(1, res.getInt(1));
-    assertEquals(1, res.getInt(2));
-    assertEquals(7706, res.getInt(3));
-    assertEquals(1, res.getInt(4));
-    assertTrue(17 == res.getFloat(5));
-    assertTrue(21168.23f == res.getFloat(6));
-    assertTrue(0.04f == res.getFloat(7));
-    assertTrue(0.02f == res.getFloat(8));
-    assertEquals("N",res.getString(9));
-    assertEquals("O",res.getString(10));
-    assertEquals("1996-03-13",res.getString(11));
-    assertEquals("1996-02-12",res.getString(12));
-    assertEquals("1996-03-22",res.getString(13));
-    assertEquals("DELIVER IN PERSON",res.getString(14));
-    assertEquals("TRUCK",res.getString(15));
-    assertEquals("egular courts above the",res.getString(16));
-
-    res.next();
-    assertEquals(1, res.getInt(1));
-    assertEquals(1, res.getInt(2));
-    assertEquals(7311, res.getInt(3));
-    assertEquals(2, res.getInt(4));
-    assertTrue(36 == res.getFloat(5));
-    assertTrue(45983.16f == res.getFloat(6));
-    assertTrue(0.09f == res.getFloat(7));
-    assertTrue(0.06f == res.getFloat(8));
-    assertEquals("N",res.getString(9));
-    assertEquals("O",res.getString(10));
-    assertEquals("1996-04-12",res.getString(11));
-    assertEquals("1996-02-28",res.getString(12));
-    assertEquals("1996-04-20",res.getString(13));
-    assertEquals("TAKE BACK RETURN",res.getString(14));
-    assertEquals("MAIL",res.getString(15));
-    assertEquals("ly final dependencies: slyly bold",res.getString(16));
+    try {
+      res.next();
+      assertEquals(1, res.getInt(1));
+      assertEquals(1, res.getInt(2));
+      assertEquals(7706, res.getInt(3));
+      assertEquals(1, res.getInt(4));
+      assertTrue(17 == res.getFloat(5));
+      assertTrue(21168.23f == res.getFloat(6));
+      assertTrue(0.04f == res.getFloat(7));
+      assertTrue(0.02f == res.getFloat(8));
+      assertEquals("N",res.getString(9));
+      assertEquals("O",res.getString(10));
+      assertEquals("1996-03-13",res.getString(11));
+      assertEquals("1996-02-12",res.getString(12));
+      assertEquals("1996-03-22",res.getString(13));
+      assertEquals("DELIVER IN PERSON",res.getString(14));
+      assertEquals("TRUCK",res.getString(15));
+      assertEquals("egular courts above the",res.getString(16));
+
+      res.next();
+      assertEquals(1, res.getInt(1));
+      assertEquals(1, res.getInt(2));
+      assertEquals(7311, res.getInt(3));
+      assertEquals(2, res.getInt(4));
+      assertTrue(36 == res.getFloat(5));
+      assertTrue(45983.16f == res.getFloat(6));
+      assertTrue(0.09f == res.getFloat(7));
+      assertTrue(0.06f == res.getFloat(8));
+      assertEquals("N",res.getString(9));
+      assertEquals("O",res.getString(10));
+      assertEquals("1996-04-12",res.getString(11));
+      assertEquals("1996-02-28",res.getString(12));
+      assertEquals("1996-04-20",res.getString(13));
+      assertEquals("TAKE BACK RETURN",res.getString(14));
+      assertEquals("MAIL",res.getString(15));
+      assertEquals("ly final dependencies: slyly bold",res.getString(16));
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -145,21 +161,29 @@ public class TestSelectQuery {
 
     ResultSet res = tpch.execute(
         "select distinct l_orderkey, l_linenumber from lineitem");
-    int cnt = 0;
-    while(res.next()) {
-      assertTrue(result1.contains(res.getInt(1) + "," + res.getInt(2)));
-      cnt++;
+    try {
+      int cnt = 0;
+      while(res.next()) {
+        assertTrue(result1.contains(res.getInt(1) + "," + res.getInt(2)));
+        cnt++;
+      }
+      assertEquals(5, cnt);
+    } finally {
+      res.close();
     }
-    assertEquals(5, cnt);
 
     res = tpch.execute("select distinct l_orderkey from lineitem");
-    Set<Integer> result2 = Sets.newHashSet(1,2,3);
-    cnt = 0;
-    while (res.next()) {
-      assertTrue(result2.contains(res.getInt(1)));
-      cnt++;
+    try {
+      Set<Integer> result2 = Sets.newHashSet(1,2,3);
+      int cnt = 0;
+      while (res.next()) {
+        assertTrue(result2.contains(res.getInt(1)));
+        cnt++;
+      }
+      assertEquals(3,cnt);
+    } finally {
+      res.close();
     }
-    assertEquals(3,cnt);
   }
 
   @Test
@@ -169,12 +193,16 @@ public class TestSelectQuery {
 
     ResultSet res = tpch.execute(
         "SELECT n_name FROM nation WHERE n_name LIKE '%IA'");
-    int cnt = 0;
-    while(res.next()) {
-      assertTrue(result.contains(res.getString(1)));
-      cnt++;
+    try {
+      int cnt = 0;
+      while(res.next()) {
+        assertTrue(result.contains(res.getString(1)));
+        cnt++;
+      }
+      assertEquals(result.size(), cnt);
+    } finally {
+      res.close();
     }
-    assertEquals(result.size(), cnt);
   }
 
   @Test
@@ -183,21 +211,29 @@ public class TestSelectQuery {
 
     ResultSet res = tpch.execute(
         "select l_orderkey from lineitem where l_shipdate <= '1996-03-22'");
-    int cnt = 0;
-    while(res.next()) {
-      assertTrue(result.contains(res.getInt(1)));
-      cnt++;
+    try {
+      int cnt = 0;
+      while(res.next()) {
+        assertTrue(result.contains(res.getInt(1)));
+        cnt++;
+      }
+      assertEquals(3, cnt);
+    } finally {
+      res.close();
     }
-    assertEquals(3, cnt);
   }
 
   @Test
   public final void testRealValueCompare() throws Exception {
     ResultSet res = tpch.execute("select ps_supplycost from partsupp where ps_supplycost = 771.64");
 
-    res.next();
-    assertTrue(771.64f == res.getFloat(1));
-    assertFalse(res.next());
+    try {
+      res.next();
+      assertTrue(771.64f == res.getFloat(1));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -211,19 +247,23 @@ public class TestSelectQuery {
             "else 'zero' " +
             "end as cond from region");
 
-    Map<Integer, String> result = Maps.newHashMap();
-    result.put(0, "zero");
-    result.put(1, "one");
-    result.put(2, "two");
-    result.put(3, "three");
-    result.put(4, "four");
-    int cnt = 0;
-    while(res.next()) {
-      assertEquals(result.get(res.getInt(1)), res.getString(2));
-      cnt++;
+    try {
+      Map<Integer, String> result = Maps.newHashMap();
+      result.put(0, "zero");
+      result.put(1, "one");
+      result.put(2, "two");
+      result.put(3, "three");
+      result.put(4, "four");
+      int cnt = 0;
+      while(res.next()) {
+        assertEquals(result.get(res.getInt(1)), res.getString(2));
+        cnt++;
+      }
+
+      assertEquals(5, cnt);
+    } finally {
+      res.close();
     }
-
-    assertEquals(5, cnt);
   }
 
   @Test
@@ -235,75 +275,98 @@ public class TestSelectQuery {
         "when r_regionkey = 4 then 'four' " +
         "end as cond from region");
 
-    Map<Integer, String> result = Maps.newHashMap();
-    result.put(0, "NULL");
-    result.put(1, "one");
-    result.put(2, "two");
-    result.put(3, "three");
-    result.put(4, "four");
-    int cnt = 0;
-    while(res.next()) {
-      assertEquals(result.get(res.getInt(1)), res.getString(2));
-      cnt++;
+    try {
+      Map<Integer, String> result = Maps.newHashMap();
+      result.put(0, "NULL");
+      result.put(1, "one");
+      result.put(2, "two");
+      result.put(3, "three");
+      result.put(4, "four");
+      int cnt = 0;
+      while(res.next()) {
+        assertEquals(result.get(res.getInt(1)), res.getString(2));
+        cnt++;
+      }
+
+      assertEquals(5, cnt);
+    } finally {
+      res.close();
     }
-
-    assertEquals(5, cnt);
   }
 
   @Test
   public final void testNotEqual() throws Exception {
     ResultSet res = tpch.execute(
         "select l_orderkey from lineitem where l_orderkey != 1");
-    assertTrue(res.next());
-    assertEquals(2, res.getInt(1));
-    assertTrue(res.next());
-    assertEquals(3, res.getInt(1));
-    assertTrue(res.next());
-    assertEquals(3, res.getInt(1));
-    assertFalse(res.next());
+    try {
+      assertTrue(res.next());
+      assertEquals(2, res.getInt(1));
+      assertTrue(res.next());
+      assertEquals(3, res.getInt(1));
+      assertTrue(res.next());
+      assertEquals(3, res.getInt(1));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testUnion1() throws Exception {
     ResultSet res = tpch.execute(
         "select o_custkey as num from orders union select c_custkey as num from customer");
-    int count = 0;
-    for (;res.next();) {
-      count++;
+    try {
+      int count = 0;
+      for (;res.next();) {
+        count++;
+      }
+      assertEquals(6, count);
+    } finally {
+      res.close();
     }
-    assertEquals(6, count);
   }
 
   @Test
   public final void testUnion2() throws Exception {
     ResultSet res = tpch.execute(
         "select l_orderkey from lineitem l1 union select l_orderkey from lineitem l2");
-    int count = 0;
-    for (;res.next();) {
-      count++;
+    try {
+      int count = 0;
+      for (;res.next();) {
+        count++;
+      }
+    } finally {
+      res.close();
     }
-    assertEquals(10, count);
   }
 
   @Test
   public final void testCreateAfterSelect() throws Exception {
     ResultSet res = tpch.execute(
         "create table orderkeys as select l_orderkey from lineitem");
-    int count = 0;
-    for (;res.next();) {
-      count++;
+    try {
+      int count = 0;
+      for (;res.next();) {
+        count++;
+      }
+      assertEquals(count, 5);
+    } finally {
+      res.close();
     }
-    assertEquals(count, 5);
   }
 
   //@Test
   // TODO - fix and enable this unit test
   public final void testLimit() throws Exception {
     ResultSet res = tpch.execute("select l_orderkey from lineitem limit 3");
-    int count = 0;
-    for (;res.next();) {
-      count++;
+    try {
+      int count = 0;
+      for (;res.next();) {
+        count++;
+      }
+      assertEquals(3, count);
+    } finally {
+      res.close();
     }
-    assertEquals(3, count);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
index 14344fe..b211bc8 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSortQuery.java
@@ -50,70 +50,86 @@ public class TestSortQuery {
   public final void testSort() throws Exception {
     ResultSet res = tpch.execute(
         "select l_linenumber, l_orderkey from lineitem order by l_orderkey");
-    int cnt = 0;
-    Long prev = null;
-    while(res.next()) {
-      if (prev == null) {
-        prev = res.getLong(2);
-      } else {
-        assertTrue(prev <= res.getLong(2));
-        prev = res.getLong(2);
+    try {
+      int cnt = 0;
+      Long prev = null;
+      while(res.next()) {
+        if (prev == null) {
+          prev = res.getLong(2);
+        } else {
+          assertTrue(prev <= res.getLong(2));
+          prev = res.getLong(2);
+        }
+        cnt++;
       }
-      cnt++;
-    }
 
-    assertEquals(5, cnt);
+      assertEquals(5, cnt);
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testSortWithAliasKey() throws Exception {
     ResultSet res = tpch.execute(
         "select l_linenumber, l_orderkey as sortkey from lineitem order by sortkey");
-    int cnt = 0;
-    Long prev = null;
-    while(res.next()) {
-      if (prev == null) {
-        prev = res.getLong(2);
-      } else {
-        assertTrue(prev <= res.getLong(2));
-        prev = res.getLong(2);
+    try {
+      int cnt = 0;
+      Long prev = null;
+      while(res.next()) {
+        if (prev == null) {
+          prev = res.getLong(2);
+        } else {
+          assertTrue(prev <= res.getLong(2));
+          prev = res.getLong(2);
+        }
+        cnt++;
       }
-      cnt++;
-    }
 
-    assertEquals(5, cnt);
+      assertEquals(5, cnt);
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testSortDesc() throws Exception {
     ResultSet res = tpch.execute(
         "select l_linenumber, l_orderkey from lineitem order by l_orderkey desc");
-    int cnt = 0;
-    Long prev = null;
-    while(res.next()) {
-      if (prev == null) {
-        prev = res.getLong(2);
-      } else {
-        assertTrue(prev >= res.getLong(2));
-        prev = res.getLong(2);
+    try {
+      int cnt = 0;
+      Long prev = null;
+      while(res.next()) {
+        if (prev == null) {
+          prev = res.getLong(2);
+        } else {
+          assertTrue(prev >= res.getLong(2));
+          prev = res.getLong(2);
+        }
+        cnt++;
       }
-      cnt++;
-    }
 
-    assertEquals(5, cnt);
+      assertEquals(5, cnt);
+    } finally {
+      res.close();
+    }
   }
 
   @Test
   public final void testTopK() throws Exception {
     ResultSet res = tpch.execute(
         "select l_orderkey, l_linenumber from lineitem order by l_orderkey desc limit 3");
-    assertTrue(res.next());
-    assertEquals(3, res.getLong(1));
-    assertTrue(res.next());
-    assertEquals(3, res.getLong(1));
-    assertTrue(res.next());
-    assertEquals(2, res.getLong(1));
-    assertFalse(res.next());
+    try {
+      assertTrue(res.next());
+      assertEquals(3, res.getLong(1));
+      assertTrue(res.next());
+      assertEquals(3, res.getLong(1));
+      assertTrue(res.next());
+      assertEquals(2, res.getLong(1));
+      assertFalse(res.next());
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -121,19 +137,23 @@ public class TestSortQuery {
     ResultSet res = tpch.execute("select max(l_quantity), l_orderkey "
         + "from lineitem group by l_orderkey order by l_orderkey");
 
-    int cnt = 0;
-    Long prev = null;
-    while(res.next()) {
-      if (prev == null) {
-        prev = res.getLong(1);
-      } else {
-        assertTrue(prev <= res.getLong(1));
-        prev = res.getLong(1);
+    try {
+      int cnt = 0;
+      Long prev = null;
+      while(res.next()) {
+        if (prev == null) {
+          prev = res.getLong(1);
+        } else {
+          assertTrue(prev <= res.getLong(1));
+          prev = res.getLong(1);
+        }
+        cnt++;
       }
-      cnt++;
-    }
 
-    assertEquals(3, cnt);
+      assertEquals(3, cnt);
+    } finally {
+      res.close();
+    }
   }
 
   @Test
@@ -141,18 +161,22 @@ public class TestSortQuery {
     ResultSet res = tpch.execute("select max(l_quantity) as max_quantity, l_orderkey "
         + "from lineitem group by l_orderkey order by max_quantity");
 
-    int cnt = 0;
-    Long prev = null;
-    while(res.next()) {
-      if (prev == null) {
-        prev = res.getLong(1);
-      } else {
-        assertTrue(prev <= res.getLong(1));
-        prev = res.getLong(1);
+    try {
+      int cnt = 0;
+      Long prev = null;
+      while(res.next()) {
+        if (prev == null) {
+          prev = res.getLong(1);
+        } else {
+          assertTrue(prev <= res.getLong(1));
+          prev = res.getLong(1);
+        }
+        cnt++;
       }
-      cnt++;
-    }
 
-    assertEquals(3, cnt);
+      assertEquals(3, cnt);
+    } finally {
+      res.close();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
index 69b8bac..a67dd26 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
@@ -72,7 +72,7 @@ public class TestExecutionBlockCursor {
     AsyncDispatcher dispatcher = new AsyncDispatcher();
     dispatcher.init(conf);
     dispatcher.start();
-    planner = new GlobalPlanner(conf, catalog, sm, dispatcher.getEventHandler());
+    planner = new GlobalPlanner(conf, sm, dispatcher.getEventHandler());
   }
 
   public static void tearDown() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 55b9a91..a8924dd 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,16 @@
 
 package org.apache.tajo.master;
 
-import org.apache.tajo.TestTajoIds;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
-import org.junit.Test;
 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;
 
 import java.net.URI;
 import java.util.*;
@@ -48,7 +50,7 @@ public class TestRepartitioner {
 
     Collection<URI> uris = Repartitioner.
         createHashFetchURL(hostName + ":" + port, sid, partitionId,
-            PartitionType.HASH, intermediateEntries);
+                PartitionType.HASH, intermediateEntries);
 
     List<String> taList = TUtil.newList();
     for (URI uri : uris) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 e61f721..05a269e 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
@@ -29,7 +29,7 @@ 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.MasterWorkerProtocol.MasterWorkerProtocolService;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
 import org.apache.tajo.rpc.ProtoAsyncRpcClient;
 import org.apache.tajo.util.TajoIdUtils;
 
@@ -46,8 +46,8 @@ public class TaskRunnerTest {
     ProtoAsyncRpcClient mockClient = mock(ProtoAsyncRpcClient.class);
     mockClient.close();
 
-    MasterWorkerProtocolService.Interface mockMaster =
-        mock(MasterWorkerProtocolService.Interface.class);
+    QueryMasterProtocolService.Interface mockMaster =
+        mock(QueryMasterProtocolService.Interface.class);
     ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
         q1.getApplicationId(), q1.getAttemptId());
     ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 1);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/log4j.properties b/tajo-core/tajo-core-backend/src/test/resources/log4j.properties
index ad07100..145703c 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/test/resources/log4j.properties
@@ -19,7 +19,10 @@
 # log4j configuration used during build and unit tests
 
 log4j.rootLogger=info,stdout
-log4j.threshhold=DEBUG
+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
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 2035a4e..1ce6aba 100644
--- a/tajo-core/tajo-core-pullserver/pom.xml
+++ b/tajo-core/tajo-core-pullserver/pom.xml
@@ -59,11 +59,10 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
-      <groupId>org.jboss.netty</groupId>
+      <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
       <scope>compile</scope>
     </dependency>
-
     <dependency>
       <groupId>commons-logging</groupId>
       <artifactId>commons-logging</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto b/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
index 96b35b3..6164553 100644
--- a/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
+++ b/tajo-core/tajo-core-storage/src/main/proto/CatalogProtos.proto
@@ -25,9 +25,9 @@ option java_generate_equals_and_hash = true;
 import "DataTypes.proto";
 
 enum StoreType {
-  MEM = 0;
-  CSV = 1;
-  RAW = 2;
+	MEM = 0;
+	CSV = 1;
+	RAW = 2;
   RCFILE = 3;
   ROWFILE = 4;
   HCFILE = 5;
@@ -35,147 +35,147 @@ enum StoreType {
 }
 
 enum OrderType {
-  ORDER_NONE = 0;
-  ASC = 1;
-  DSC = 2;
+    ORDER_NONE = 0;
+    ASC = 1;
+    DSC = 2;
 }
 
 enum CompressType {
-  COMP_NONE = 0;
-  NULL_SUPPRESS = 1;
-  RUN_LENGTH = 2;
-  BIT_VECTOR = 3;
-  DICTIONARY = 4;
-  SNAPPY = 5;
-  LZ = 6;
+    COMP_NONE = 0;
+    NULL_SUPPRESS = 1;
+    RUN_LENGTH = 2;
+    BIT_VECTOR = 3;
+    DICTIONARY = 4;
+    SNAPPY = 5;
+    LZ = 6;
 }
 
 message ColumnMetaProto {
-  required DataType dataType = 1;
-  required bool compressed = 2;
-  required bool sorted = 3;
-  required bool contiguous = 4;
-  required StoreType storeType = 5;
-  required CompressType compType = 6;
-  required int64 startRid = 7;
-  required int32 recordNum = 8;
-  required int32 offsetToIndex = 9;
+    required DataType dataType = 1;
+    required bool compressed = 2;
+    required bool sorted = 3;
+    required bool contiguous = 4;
+    required StoreType storeType = 5;
+    required CompressType compType = 6;
+    required int64 startRid = 7;
+    required int32 recordNum = 8;
+    required int32 offsetToIndex = 9;
 }
 
 message ColumnProto {
-  required string columnName = 1;
-  required DataType dataType = 2;
+	required string columnName = 1;
+	required DataType dataType = 2;
 }
 
 message SchemaProto {
-  repeated ColumnProto fields = 1;
+	repeated ColumnProto fields = 1;
 }
 
 message KeyValueProto {
-  required string key = 1;
-  required string value = 2;
+	required string key = 1;
+	required string value = 2;
 }
 
 message KeyValueSetProto {
-  repeated KeyValueProto keyval = 1;
+	repeated KeyValueProto keyval = 1;
 }
 
 message FragmentProto {
-  required string id = 1;
-  required string path = 2;
-  required int64 startOffset = 3;
-  required int64 length = 4;
-  required TableProto meta = 5;
-  optional TableStatProto stat = 6;
+	required string id = 1;
+	required string path = 2;
+	required int64 startOffset = 3;
+	required int64 length = 4;
+	required TableProto meta = 5;
+	optional TableStatProto stat = 6;
   optional bool distCached = 7 [default = false];
 }
 
 message TableProto {
-  required SchemaProto schema = 1;
-  required StoreType storeType = 2;
-  required KeyValueSetProto params = 3;
-  optional TableStatProto stat = 4;
+    required SchemaProto schema = 1;
+    required StoreType storeType = 2;
+    required KeyValueSetProto params = 3;
+    optional TableStatProto stat = 4;
 }
 
 message TableDescProto {
-  required string id = 1;
-  required string path = 2;
-  required TableProto meta = 3;
+	required string id = 1;
+	required string path = 2;
+	required TableProto meta = 3;
 }
 
 enum FunctionType {
-  GENERAL = 0;
-  AGGREGATION = 1;
+	GENERAL = 0;
+	AGGREGATION = 1;
 }
 
 message FunctionDescProto {
-  required string signature = 1;
-  required string className = 2;
-  required FunctionType type = 3;
-  repeated Type parameterTypes = 4;
-  required Type returnType = 5;
+	required string signature = 1;
+	required string className = 2;
+	required FunctionType type = 3;
+	repeated DataType parameterTypes = 4;
+	required DataType returnType = 5;
 }
 
 message IndexDescProto {
-  required string name = 1;
-  required string tableId = 2;
-  required ColumnProto column = 3;
-  required IndexMethod indexMethod = 4;
-  optional bool isUnique = 5 [default = false];
-  optional bool isClustered = 6 [default = false];
-  optional bool isAscending = 7 [default = false];
+    required string name = 1;
+    required string tableId = 2;
+    required ColumnProto column = 3;
+    required IndexMethod indexMethod = 4;
+    optional bool isUnique = 5 [default = false];
+    optional bool isClustered = 6 [default = false];
+    optional bool isAscending = 7 [default = false];
 }
 
 enum IndexMethod {
-  TWO_LEVEL_BIN_TREE = 0;
-  BTREE = 1;
-  HASH = 2;
-  BITMAP = 3;
+    TWO_LEVEL_BIN_TREE = 0;
+    BTREE = 1;
+    HASH = 2;
+    BITMAP = 3;
 }
 
 message GetAllTableNamesResponse {
-  repeated string tableName = 1;
+    repeated string tableName = 1;
 }
 
 message GetIndexRequest {
-  required string tableName = 1;
-  required string columnName = 2;
+    required string tableName = 1;
+    required string columnName = 2;
 }
 
 message GetFunctionsResponse {
-  repeated FunctionDescProto functionDesc = 1;
+	repeated FunctionDescProto functionDesc = 1;
 }
 
 message UnregisterFunctionRequest {
-  required string signature = 1;
-  repeated Type parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message GetFunctionMetaRequest {
-  required string signature = 1;
-  repeated Type parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message ContainFunctionRequest {
-  required string signature = 1;
-  repeated Type parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message TableStatProto {
-  required int64 numRows = 1;
-  required int64 numBytes = 2;
-  optional int32 numBlocks = 3;
-  optional int32 numPartitions = 4;
-  optional int64 avgRows = 5;
-  repeated ColumnStatProto colStat = 6;
+	required int64 numRows = 1;
+	required int64 numBytes = 2;
+	optional int32 numBlocks = 3;
+	optional int32 numPartitions = 4;
+	optional int64 avgRows = 5;
+	repeated ColumnStatProto colStat = 6;
 }
 
 message ColumnStatProto {
-  required ColumnProto column = 1;
-  optional int64 numDistVal = 2;
-  optional int64 numNulls = 3;
-  optional bytes minValue = 4;
-  optional bytes maxValue = 5;
+    required ColumnProto column = 1;
+    optional int64 numDistVal = 2;
+    optional int64 numNulls = 3;
+    optional bytes minValue = 4;
+    optional bytes maxValue = 5;
 }
 
 enum StatType {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index b03880e..068623a 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -36,7 +36,7 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
     <tajo.version>0.2.0-SNAPSHOT</tajo.version>
-    <hadoop.version>2.0.3-alpha</hadoop.version>
+    <hadoop.version>2.0.5-alpha</hadoop.version>
   </properties>
 
   <licenses>
@@ -749,11 +749,18 @@
         <version>1.9.5-rc1</version>
         <scope>test</scope>
       </dependency>
+      <!--
       <dependency>
         <groupId>org.jboss.netty</groupId>
         <artifactId>netty</artifactId>
         <version>3.2.4.Final</version>
       </dependency>
+      -->
+      <dependency>
+        <groupId>io.netty</groupId>
+        <artifactId>netty</artifactId>
+        <version>3.6.6.Final</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
   <profiles>

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-rpc/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-rpc/pom.xml b/tajo-rpc/pom.xml
index 2c28282..6875291 100644
--- a/tajo-rpc/pom.xml
+++ b/tajo-rpc/pom.xml
@@ -135,8 +135,9 @@
       <artifactId>protobuf-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.jboss.netty</groupId>
+      <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
+      <scope>compile</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>


[6/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Repartitioner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Repartitioner.java
deleted file mode 100644
index d97072d..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/Repartitioner.java
+++ /dev/null
@@ -1,582 +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.fs.Path;
-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;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.planner.PlannerUtil;
-import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
-import org.apache.tajo.engine.planner.UniformRangePartition;
-import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.engine.utils.TupleUtil;
-import org.apache.tajo.exception.InternalException;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
-import org.apache.tajo.master.QueryUnit.IntermediateEntry;
-import org.apache.tajo.storage.Fragment;
-import org.apache.tajo.storage.TupleRange;
-import org.apache.tajo.util.TUtil;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.math.BigDecimal;
-import java.net.URI;
-import java.util.*;
-import java.util.Map.Entry;
-
-/**
- * Repartitioner creates non-leaf tasks and shuffles intermediate data.
- * It supports two repartition methods, such as hash and range repartition.
- */
-public class Repartitioner {
-  private static final Log LOG = LogFactory.getLog(Repartitioner.class);
-
-  private static int HTTP_REQUEST_MAXIMUM_LENGTH = 1900;
-
-  public static QueryUnit [] createJoinTasks(SubQuery subQuery)
-      throws IOException {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    CatalogService catalog = subQuery.getContext().getCatalog();
-
-    ScanNode[] scans = execBlock.getScanNodes();
-    Path tablePath;
-    Fragment [] fragments = new Fragment[2];
-    TableStat [] stats = new TableStat[2];
-
-    // initialize variables from the child operators
-    for (int i =0; i < 2; i++) {
-      // TODO - temporarily tables should be stored in temporarily catalog for each query
-      if (scans[i].getTableId().startsWith(SubQueryId.PREFIX)) {
-        tablePath = subQuery.getStorageManager().getTablePath(scans[i].getTableId());
-      } else {
-        tablePath = catalog.getTableDesc(scans[i].getTableId()).getPath();
-      }
-
-      if (scans[i].isLocal()) { // it only requires a dummy fragment.
-        fragments[i] = new Fragment(scans[i].getTableId(), tablePath,
-            CatalogUtil.newTableMeta(scans[i].getInSchema(), StoreType.CSV),
-            0, 0, null);
-      } else {
-        fragments[i] = subQuery.getStorageManager().getSplits(scans[i].getTableId(),
-            catalog.getTableDesc(scans[i].getTableId()).getMeta(),
-            new Path(tablePath, "data")).get(0);
-      }
-
-      // Getting a table stat for each scan
-      stats[i] = subQuery.getChildQuery(scans[i]).getTableStat();
-    }
-
-    // Assigning either fragments or fetch urls to query units
-    QueryUnit [] tasks;
-    if (scans[0].isBroadcast() || scans[1].isBroadcast()) {
-      tasks = new QueryUnit[1];
-      tasks[0] = new QueryUnit(QueryIdFactory.newQueryUnitId(subQuery.getId(), 0),
-          false, subQuery.getEventHandler());
-      tasks[0].setLogicalPlan(execBlock.getPlan());
-      tasks[0].setFragment(scans[0].getTableId(), fragments[0]);
-      tasks[0].setFragment(scans[1].getTableId(), fragments[1]);
-    } else {
-      // The hash map is modeling as follows:
-      // <Partition Id, <Table Name, Intermediate Data>>
-      Map<Integer, Map<String, List<IntermediateEntry>>> hashEntries =
-          new HashMap<Integer, Map<String, List<IntermediateEntry>>>();
-
-      // Grouping IntermediateData by a partition key and a table name
-      for (ScanNode scan : scans) {
-        SubQuery childSubQuery = subQuery.getChildQuery(scan);
-        for (QueryUnit task : childSubQuery.getQueryUnits()) {
-          if (task.getIntermediateData() != null) {
-            for (IntermediateEntry intermEntry : task.getIntermediateData()) {
-              if (hashEntries.containsKey(intermEntry.getPartitionId())) {
-                Map<String, List<IntermediateEntry>> tbNameToInterm =
-                    hashEntries.get(intermEntry.getPartitionId());
-
-                if (tbNameToInterm.containsKey(scan.getTableId())) {
-                  tbNameToInterm.get(scan.getTableId()).add(intermEntry);
-                } else {
-                  tbNameToInterm.put(scan.getTableId(), TUtil.newList(intermEntry));
-                }
-              } else {
-                Map<String, List<IntermediateEntry>> tbNameToInterm =
-                    new HashMap<String, List<IntermediateEntry>>();
-                tbNameToInterm.put(scan.getTableId(), TUtil.newList(intermEntry));
-                hashEntries.put(intermEntry.getPartitionId(), tbNameToInterm);
-              }
-            }
-          }
-        }
-      }
-
-      LOG.info("Outer Intermediate Volume: " + stats[0].getNumBytes());
-      LOG.info("Inner Intermediate Volume: " + stats[1].getNumBytes());
-
-      // Getting the desire number of join tasks according to the volumn
-      // of a larger table
-      int largerIdx = stats[0].getNumBytes() >= stats[1].getNumBytes() ? 0 : 1;
-      int desireJoinTaskVolumn = subQuery.getContext().getConf().
-          getIntVar(ConfVars.JOIN_TASK_VOLUME);
-
-      // calculate the number of tasks according to the data size
-      int mb = (int) Math.ceil((double)stats[largerIdx].getNumBytes() / 1048576);
-      LOG.info("Larger intermediate data is approximately " + mb + " MB");
-      // determine the number of task per 64MB
-      int maxTaskNum = (int) Math.ceil((double)mb / desireJoinTaskVolumn);
-      LOG.info("The calculated number of tasks is " + maxTaskNum);
-      LOG.info("The number of total partition keys is " + hashEntries.size());
-      // the number of join tasks cannot be larger than the number of
-      // distinct partition ids.
-      int joinTaskNum = Math.min(maxTaskNum, hashEntries.size());
-      LOG.info("The determined number of join tasks is " + joinTaskNum);
-      QueryUnit [] createdTasks = newEmptyJoinTask(subQuery, fragments, joinTaskNum);
-
-      // Assign partitions to tasks in a round robin manner.
-      int i = 0;
-      for (Entry<Integer, Map<String, List<IntermediateEntry>>> entry
-          : hashEntries.entrySet()) {
-        addJoinPartition(createdTasks[i++], subQuery, entry.getKey(), entry.getValue());
-        if (i >= joinTaskNum) {
-          i = 0;
-        }
-      }
-
-      List<QueryUnit> filteredTasks = new ArrayList<QueryUnit>();
-      for (QueryUnit task : createdTasks) {
-        // if there are at least two fetches, the join is possible.
-        if (task.getFetches().size() > 1) {
-          filteredTasks.add(task);
-        }
-      }
-
-      tasks = filteredTasks.toArray(new QueryUnit[filteredTasks.size()]);
-    }
-
-    return tasks;
-  }
-
-  private static QueryUnit [] newEmptyJoinTask(SubQuery subQuery, Fragment [] fragments, int taskNum) {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    QueryUnit [] tasks = new QueryUnit[taskNum];
-    for (int i = 0; i < taskNum; i++) {
-      tasks[i] = new QueryUnit(
-          QueryIdFactory.newQueryUnitId(subQuery.getId(), i), execBlock.isLeafBlock(),
-          subQuery.getEventHandler());
-      tasks[i].setLogicalPlan(execBlock.getPlan());
-      for (Fragment fragment : fragments) {
-        tasks[i].setFragment2(fragment);
-      }
-    }
-
-    return tasks;
-  }
-
-  private static void addJoinPartition(QueryUnit task, SubQuery subQuery, int partitionId,
-                                       Map<String, List<IntermediateEntry>> grouppedPartitions) {
-
-    for (ScanNode scanNode : subQuery.getBlock().getScanNodes()) {
-      Map<String, List<IntermediateEntry>> requests;
-      if (grouppedPartitions.containsKey(scanNode.getTableId())) {
-          requests = mergeHashPartitionRequest(grouppedPartitions.get(scanNode.getTableId()));
-      } else {
-        return;
-      }
-      Set<URI> fetchURIs = TUtil.newHashSet();
-      for (Entry<String, List<IntermediateEntry>> requestPerNode : requests.entrySet()) {
-        Collection<URI> uris = createHashFetchURL(requestPerNode.getKey(),
-            subQuery.getChildQuery(scanNode).getId(),
-            partitionId, PartitionType.HASH,
-            requestPerNode.getValue());
-        fetchURIs.addAll(uris);
-      }
-      task.addFetches(scanNode.getTableId(), fetchURIs);
-    }
-  }
-
-  /**
-   * This method merges the partition request associated with the pullserver's address.
-   * It reduces the number of TCP connections.
-   *
-   * @return key: pullserver's address, value: a list of requests
-   */
-  private static Map<String, List<IntermediateEntry>> mergeHashPartitionRequest(
-      List<IntermediateEntry> partitions) {
-    Map<String, List<IntermediateEntry>> mergedPartitions =
-        new HashMap<String, List<IntermediateEntry>>();
-    for (IntermediateEntry partition : partitions) {
-      if (mergedPartitions.containsKey(partition.getPullAddress())) {
-        mergedPartitions.get(partition.getPullAddress()).add(partition);
-      } else {
-        mergedPartitions.put(partition.getPullAddress(), TUtil.newList(partition));
-      }
-    }
-
-    return mergedPartitions;
-  }
-
-  public static QueryUnit [] createNonLeafTask(SubQuery subQuery,
-                                               SubQuery childSubQuery,
-                                               int maxNum)
-      throws InternalException {
-    ExecutionBlock childExecBlock = childSubQuery.getBlock();
-    if (childExecBlock.getPartitionType() == PartitionType.HASH) {
-      return createHashPartitionedTasks(subQuery, childSubQuery, maxNum);
-    } else if (childExecBlock.getPartitionType() == PartitionType.RANGE) {
-      return createRangePartitionedTasks(subQuery, childSubQuery, maxNum);
-    } else {
-      throw new InternalException("Cannot support partition type");
-    }
-  }
-
-  public static QueryUnit [] createRangePartitionedTasks(SubQuery subQuery,
-                                                         SubQuery childSubQuery,
-                                                         int maxNum)
-      throws InternalException {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    TableStat stat = childSubQuery.getTableStat();
-    if (stat.getNumRows() == 0) {
-      return new QueryUnit[0];
-    }
-
-    ScanNode scan = execBlock.getScanNodes()[0];
-    Path tablePath;
-    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableId());
-
-    StoreTableNode store = (StoreTableNode) childSubQuery.getBlock().getPlan();
-    SortNode sort = (SortNode) store.getSubNode();
-    SortSpec[] sortSpecs = sort.getSortKeys();
-    Schema sortSchema = PlannerUtil.sortSpecsToSchema(sort.getSortKeys());
-
-    // calculate the number of maximum query ranges
-    TupleRange mergedRange =
-        TupleUtil.columnStatToRange(sort.getOutSchema(),
-            sortSchema, stat.getColumnStats());
-    RangePartitionAlgorithm partitioner =
-        new UniformRangePartition(sortSchema, mergedRange);
-    BigDecimal card = partitioner.getTotalCardinality();
-
-    // if the number of the range cardinality is less than the desired number of tasks,
-    // we set the the number of tasks to the number of range cardinality.
-    int determinedTaskNum;
-    if (card.compareTo(new BigDecimal(maxNum)) < 0) {
-      LOG.info("The range cardinality (" + card
-          + ") is less then the desired number of tasks (" + maxNum + ")");
-      determinedTaskNum = card.intValue();
-    } else {
-      determinedTaskNum = maxNum;
-    }
-
-    LOG.info("Try to divide " + mergedRange + " into " + determinedTaskNum +
-        " sub ranges (total units: " + determinedTaskNum + ")");
-    TupleRange [] ranges = partitioner.partition(determinedTaskNum);
-
-    Fragment dummyFragment = new Fragment(scan.getTableId(), tablePath,
-        CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
-        0, 0, null);
-
-    List<String> basicFetchURIs = new ArrayList<String>();
-
-    SubQuery child = childSubQuery.getContext().getSubQuery(
-        subQuery.getBlock().getChildBlock(scan).getId());
-    for (QueryUnit qu : child.getQueryUnits()) {
-      for (IntermediateEntry p : qu.getIntermediateData()) {
-        String uri = createBasicFetchUri(p.getPullHost(), p.getPullPort(),
-            childSubQuery.getId(), p.taskId, p.attemptId);
-        basicFetchURIs.add(uri);
-      }
-    }
-
-    boolean ascendingFirstKey = sortSpecs[0].isAscending();
-    SortedMap<TupleRange, Set<URI>> map;
-    if (ascendingFirstKey) {
-      map = new TreeMap<TupleRange, Set<URI>>();
-    } else {
-      map = new TreeMap<TupleRange, Set<URI>>(new TupleRange.DescendingTupleRangeComparator());
-    }
-
-    Set<URI> uris;
-    try {
-      for (int i = 0; i < ranges.length; i++) {
-        uris = new HashSet<URI>();
-        for (String uri: basicFetchURIs) {
-          String rangeParam = TupleUtil.rangeToQuery(sortSchema, ranges[i],
-              ascendingFirstKey, ascendingFirstKey ? i == (ranges.length - 1) : i == 0);
-          URI finalUri = URI.create(uri + "&" + rangeParam);
-          uris.add(finalUri);
-        }
-        map.put(ranges[i], uris);
-      }
-
-    } catch (UnsupportedEncodingException e) {
-      LOG.error(e);
-    }
-
-    QueryUnit [] tasks = createEmptyNonLeafTasks(subQuery, determinedTaskNum, dummyFragment);
-    assignPartitionByRoundRobin(map, scan.getTableId(), tasks);
-    return tasks;
-  }
-
-  public static QueryUnit [] assignPartitionByRoundRobin(Map<?, Set<URI>> partitions,
-                                               String tableName, QueryUnit [] tasks) {
-    int tid = 0;
-    for (Entry<?, Set<URI>> entry : partitions.entrySet()) {
-      for (URI uri : entry.getValue()) {
-        tasks[tid].addFetch(tableName, uri);
-      }
-
-      if (tid >= tasks.length) {
-        tid = 0;
-      } else {
-        tid ++;
-      }
-    }
-
-    return tasks;
-  }
-
-  public static String createBasicFetchUri(String hostName, int port,
-                                           SubQueryId 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())
-        .append("&").append("ta=").append(taskId).append("_").append(attemptId)
-        .append("&").append("p=0")
-        .append("&").append("type=r");
-
-    return sb.toString();
-  }
-
-  public static QueryUnit [] createHashPartitionedTasks(SubQuery subQuery,
-                                                 SubQuery childSubQuery,
-                                                 int maxNum) {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    TableStat stat = childSubQuery.getTableStat();
-    if (stat.getNumRows() == 0) {
-      return new QueryUnit[0];
-    }
-
-    ScanNode scan = execBlock.getScanNodes()[0];
-    Path tablePath;
-    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableId());
-
-    List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
-    for (QueryUnit tasks : childSubQuery.getQueryUnits()) {
-      if (tasks.getIntermediateData() != null) {
-        partitions.addAll(tasks.getIntermediateData());
-      }
-    }
-
-    Fragment frag = new Fragment(scan.getTableId(), tablePath,
-        CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
-        0, 0, null);
-
-    Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
-    Map<String, List<IntermediateEntry>> hashedByHost;
-    Map<Integer, List<URI>> finalFetchURI = new HashMap<Integer, List<URI>>();
-
-    for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
-      hashedByHost = hashByHost(interm.getValue());
-      for (Entry<String, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
-        Collection<URI> uris = createHashFetchURL(e.getKey(), childSubQuery.getId(),
-            interm.getKey(),
-            childSubQuery.getBlock().getPartitionType(), e.getValue());
-
-        if (finalFetchURI.containsKey(interm.getKey())) {
-          finalFetchURI.get(interm.getKey()).addAll(uris);
-        } else {
-          finalFetchURI.put(interm.getKey(), TUtil.newList(uris));
-        }
-      }
-    }
-
-    GroupbyNode groupby = (GroupbyNode) childSubQuery.getBlock().getStoreTableNode().
-        getSubNode();
-    // the number of tasks cannot exceed the number of merged fetch uris.
-    int determinedTaskNum = Math.min(maxNum, finalFetchURI.size());
-    if (groupby.getGroupingColumns().length == 0) {
-      determinedTaskNum = 1;
-    }
-
-    QueryUnit [] tasks = createEmptyNonLeafTasks(subQuery, determinedTaskNum, frag);
-
-    int tid = 0;
-    for (Entry<Integer, List<URI>> entry : finalFetchURI.entrySet()) {
-      for (URI uri : entry.getValue()) {
-        tasks[tid].addFetch(scan.getTableId(), uri);
-      }
-
-      tid ++;
-
-      if (tid == tasks.length) {
-       tid = 0;
-      }
-    }
-
-    return tasks;
-  }
-
-  public static Collection<URI> createHashFetchURL(String hostAndPort, SubQueryId childSid,
-                                       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=");
-    if (type == PartitionType.HASH) {
-      urlPrefix.append("h");
-    } else if (type == PartitionType.RANGE) {
-      urlPrefix.append("r");
-    }
-    urlPrefix.append("&ta=");
-
-    // If the get request is longer than 2000 characters,
-    // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long.
-    // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15
-    // The below code transforms a long request to multiple requests.
-    List<String> taskIdsParams = new ArrayList<String>();
-    boolean first = true;
-    StringBuilder taskIdListBuilder = new StringBuilder();
-    for (IntermediateEntry entry: entries) {
-      StringBuilder taskAttemptId = new StringBuilder();
-
-      if (!first) { // when comma is added?
-        taskAttemptId.append(",");
-      } else {
-        first = false;
-      }
-
-      taskAttemptId.append(entry.getTaskId()).append("_").
-          append(entry.getAttemptId());
-      if (taskIdListBuilder.length() + taskAttemptId.length()
-          > HTTP_REQUEST_MAXIMUM_LENGTH) {
-        taskIdsParams.add(taskIdListBuilder.toString());
-        taskIdListBuilder = new StringBuilder(entry.getTaskId() + "_" + entry.getAttemptId());
-      } else {
-        taskIdListBuilder.append(taskAttemptId);
-      }
-    }
-
-    // if the url params remain
-    if (taskIdListBuilder.length() > 0) {
-      taskIdsParams.add(taskIdListBuilder.toString());
-    }
-
-    Collection<URI> fetchURLs = new ArrayList<URI>();
-    for (String param : taskIdsParams) {
-      fetchURLs.add(URI.create(urlPrefix + param));
-    }
-
-    return fetchURLs;
-  }
-
-  public static Map<Integer, List<IntermediateEntry>> hashByKey(
-      List<IntermediateEntry> entries) {
-    Map<Integer, List<IntermediateEntry>> hashed = new HashMap<Integer, List<IntermediateEntry>>();
-    for (IntermediateEntry entry : entries) {
-      if (hashed.containsKey(entry.getPartitionId())) {
-        hashed.get(entry.getPartitionId()).add(entry);
-      } else {
-        hashed.put(entry.getPartitionId(), TUtil.newList(entry));
-      }
-    }
-
-    return hashed;
-  }
-
-  public static QueryUnit [] createEmptyNonLeafTasks(SubQuery subQuery, int num,
-                                                     Fragment frag) {
-    LogicalNode plan = subQuery.getBlock().getPlan();
-    QueryUnit [] tasks = new QueryUnit[num];
-    for (int i = 0; i < num; i++) {
-      tasks[i] = new QueryUnit(QueryIdFactory.newQueryUnitId(subQuery.getId(), i),
-          false, subQuery.getEventHandler());
-      tasks[i].setFragment2(frag);
-      tasks[i].setLogicalPlan(plan);
-    }
-    return tasks;
-  }
-
-  public static Map<String, List<IntermediateEntry>> hashByHost(
-      List<IntermediateEntry> entries) {
-    Map<String, List<IntermediateEntry>> hashed = new HashMap<String, List<IntermediateEntry>>();
-
-    String hostName;
-    for (IntermediateEntry entry : entries) {
-      hostName = entry.getPullHost() + ":" + entry.getPullPort();
-      if (hashed.containsKey(hostName)) {
-        hashed.get(hostName).add(entry);
-      } else {
-        hashed.put(hostName, TUtil.newList(entry));
-      }
-    }
-
-    return hashed;
-  }
-
-  public static SubQuery setPartitionNumberForTwoPhase(SubQuery subQuery, final int n) {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    Column[] keys = null;
-    // if the next query is join,
-    // set the partition number for the current logicalUnit
-    // TODO: the union handling is required when a join has unions as its child
-    ExecutionBlock parentBlock = execBlock.getParentBlock();
-    if (parentBlock != null) {
-      if (parentBlock.getStoreTableNode().getSubNode().getType() == ExprType.JOIN) {
-        execBlock.getStoreTableNode().setPartitions(execBlock.getPartitionType(),
-            execBlock.getStoreTableNode().getPartitionKeys(), n);
-        keys = execBlock.getStoreTableNode().getPartitionKeys();
-      }
-    }
-
-    StoreTableNode store = execBlock.getStoreTableNode();
-    // set the partition number for group by and sort
-    if (execBlock.getPartitionType() == PartitionType.HASH) {
-      if (store.getSubNode().getType() == ExprType.GROUP_BY) {
-        GroupbyNode groupby = (GroupbyNode)store.getSubNode();
-        keys = groupby.getGroupingColumns();
-      }
-    } else if (execBlock.getPartitionType() == PartitionType.RANGE) {
-      if (store.getSubNode().getType() == ExprType.SORT) {
-        SortNode sort = (SortNode)store.getSubNode();
-        keys = new Column[sort.getSortKeys().length];
-        for (int i = 0; i < keys.length; i++) {
-          keys[i] = sort.getSortKeys()[i].getSortKey();
-        }
-      }
-    }
-    if (keys != null) {
-      if (keys.length == 0) {
-        store.setPartitions(execBlock.getPartitionType(), new Column[]{}, 1);
-      } else {
-        store.setPartitions(execBlock.getPartitionType(), keys, n);
-      }
-    } else {
-      store.setListPartition();
-    }
-    return subQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQuery.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQuery.java
deleted file mode 100644
index ffb7e56..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQuery.java
+++ /dev/null
@@ -1,766 +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 com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-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.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.Priority;
-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.QueryIdFactory;
-import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.catalog.CatalogService;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.statistics.ColumnStat;
-import org.apache.tajo.catalog.statistics.StatisticsUtil;
-import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.PlannerUtil;
-import org.apache.tajo.engine.planner.logical.ExprType;
-import org.apache.tajo.engine.planner.logical.GroupbyNode;
-import org.apache.tajo.engine.planner.logical.ScanNode;
-import org.apache.tajo.engine.planner.logical.StoreTableNode;
-import org.apache.tajo.master.QueryMaster.QueryContext;
-import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.storage.Fragment;
-import org.apache.tajo.storage.StorageManager;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-
-
-/**
- * SubQuery plays a role in controlling an ExecutionBlock and is a finite state machine.
- */
-public class SubQuery implements EventHandler<SubQueryEvent> {
-
-  private static final Log LOG = LogFactory.getLog(SubQuery.class);
-
-  private ExecutionBlock block;
-  private int priority;
-  private TableMeta meta;
-  private EventHandler eventHandler;
-  private final StorageManager sm;
-  private TaskSchedulerImpl taskScheduler;
-  private QueryContext context;
-
-  private long startTime;
-  private long finishTime;
-
-  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;
-
-  private StateMachineFactory<SubQuery, SubQueryState,
-      SubQueryEventType, SubQueryEvent> stateMachineFactory =
-      new StateMachineFactory <SubQuery, SubQueryState,
-          SubQueryEventType, SubQueryEvent> (SubQueryState.NEW)
-
-          .addTransition(SubQueryState.NEW,
-              EnumSet.of(SubQueryState.INIT, SubQueryState.FAILED, SubQueryState.SUCCEEDED),
-              SubQueryEventType.SQ_INIT, new InitAndRequestContainer())
-
-          .addTransition(SubQueryState.INIT, SubQueryState.CONTAINER_ALLOCATED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
-
-          .addTransition(SubQueryState.CONTAINER_ALLOCATED,
-              EnumSet.of(SubQueryState.RUNNING, SubQueryState.FAILED,
-                  SubQueryState.SUCCEEDED), SubQueryEventType.SQ_START, new StartTransition())
-          .addTransition(SubQueryState.CONTAINER_ALLOCATED, SubQueryState.CONTAINER_ALLOCATED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
-
-          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
-          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING, SubQueryEventType.SQ_START)
-          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
-              SubQueryEventType.SQ_TASK_COMPLETED, new TaskCompletedTransition())
-          .addTransition(SubQueryState.RUNNING, SubQueryState.SUCCEEDED,
-              SubQueryEventType.SQ_SUBQUERY_COMPLETED, new SubQueryCompleteTransition())
-          .addTransition(SubQueryState.RUNNING, SubQueryState.FAILED,
-              SubQueryEventType.SQ_FAILED, new InternalErrorTransition())
-
-          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
-              SubQueryEventType.SQ_START)
-          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED)
-
-          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
-              SubQueryEventType.SQ_START)
-          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
-              SubQueryEventType.SQ_CONTAINER_ALLOCATED)
-          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
-                 SubQueryEventType.SQ_FAILED)
-          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
-              SubQueryEventType.SQ_INTERNAL_ERROR);
-
-
-  private final Lock readLock;
-  private final Lock writeLock;
-
-  private int completedTaskCount = 0;
-
-  public SubQuery(QueryContext context, ExecutionBlock block, StorageManager sm) {
-    this.context = context;
-    this.block = block;
-    this.sm = sm;
-    this.eventHandler = context.getEventHandler();
-
-    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
-    this.readLock = readWriteLock.readLock();
-    this.writeLock = readWriteLock.writeLock();
-    stateMachine = stateMachineFactory.make(this);
-  }
-
-  public QueryContext getContext() {
-    return context;
-  }
-
-  public EventHandler getEventHandler() {
-    return eventHandler;
-  }
-
-  public TaskScheduler getTaskScheduler() {
-    return taskScheduler;
-  }
-
-  public void setStartTime() {
-    startTime = context.getClock().getTime();
-  }
-
-  @SuppressWarnings("UnusedDeclaration")
-  public long getStartTime() {
-    return this.startTime;
-  }
-
-  public void setFinishTime() {
-    finishTime = context.getClock().getTime();
-  }
-
-  @SuppressWarnings("UnusedDeclaration")
-  public long getFinishTime() {
-    return this.finishTime;
-  }
-
-  public float getProgress() {
-    readLock.lock();
-    try {
-      if (getState() == SubQueryState.NEW) {
-        return 0;
-      } else {
-        if (completedTaskCount == 0) {
-          return 0.0f;
-        } else {
-          return (float)completedTaskCount / (float)tasks.size();
-        }
-      }
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  public ExecutionBlock getBlock() {
-    return block;
-  }
-
-  public void addTask(QueryUnit task) {
-    tasks.put(task.getId(), task);
-  }
-
-  public void abortSubQuery(SubQueryState finalState) {
-    // TODO -
-    // - committer.abortSubQuery(...)
-    // - record SubQuery Finish Time
-    // - CleanUp Tasks
-    // - Record History
-
-    eventHandler.handle(new SubQueryCompletedEvent(getId(), finalState));
-  }
-
-  public StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent> getStateMachine() {
-    return this.stateMachine;
-  }
-
-  public void setPriority(int priority) {
-    this.priority = priority;
-  }
-
-
-  public int getPriority() {
-    return this.priority;
-  }
-
-  public StorageManager getStorageManager() {
-    return sm;
-  }
-  
-  public SubQuery getChildQuery(ScanNode scanForChild) {
-    return context.getSubQuery(block.getChildBlock(scanForChild).getId());
-  }
-  
-  public SubQueryId getId() {
-    return block.getId();
-  }
-  
-  public QueryUnit[] getQueryUnits() {
-    return tasks.values().toArray(new QueryUnit[tasks.size()]);
-  }
-  
-  public QueryUnit getQueryUnit(QueryUnitId qid) {
-    return tasks.get(qid);
-  }
-
-  public void setTableMeta(TableMeta meta) {
-    this.meta = meta;
-  }
-
-  @SuppressWarnings("UnusedDeclaration")
-  public TableMeta getTableMeta() {
-    return meta;
-  }
-
-  public TableStat getTableStat() {
-    return this.meta.getStat();
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(this.getId());
-    return sb.toString();
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof SubQuery) {
-      SubQuery other = (SubQuery)o;
-      return getId().equals(other.getId());
-    }
-    return false;
-  }
-  
-  @Override
-  public int hashCode() {
-    return getId().hashCode();
-  }
-  
-  public int compareTo(SubQuery other) {
-    return getId().compareTo(other.getId());
-  }
-
-  public SubQueryState getState() {
-    readLock.lock();
-    try {
-      return stateMachine.getCurrentState();
-    } finally {
-      readLock.unlock();
-    }
-  }
-
-  private static TableStat computeStatFromUnionBlock(SubQuery unit) {
-    TableStat stat = new TableStat();
-    TableStat childStat;
-    long avgRows = 0, numBytes = 0, numRows = 0;
-    int numBlocks = 0, numPartitions = 0;
-    List<ColumnStat> columnStats = Lists.newArrayList();
-
-    Iterator<ExecutionBlock> it = unit.getBlock().getChildBlocks().iterator();
-    while (it.hasNext()) {
-      ExecutionBlock block = it.next();
-      SubQuery childSubQuery = unit.context.getSubQuery(block.getId());
-      childStat = childSubQuery.getTableStat();
-      avgRows += childStat.getAvgRows();
-      columnStats.addAll(childStat.getColumnStats());
-      numBlocks += childStat.getNumBlocks();
-      numBytes += childStat.getNumBytes();
-      numPartitions += childStat.getNumPartitions();
-      numRows += childStat.getNumRows();
-    }
-
-    stat.setColumnStats(columnStats);
-    stat.setNumBlocks(numBlocks);
-    stat.setNumBytes(numBytes);
-    stat.setNumPartitions(numPartitions);
-    stat.setNumRows(numRows);
-    stat.setAvgRows(avgRows);
-    return stat;
-  }
-
-  public TableMeta buildTableMeta() throws IOException {
-    finishTime = context.getClock().getTime();
-
-    TableStat stat;
-    if (block.hasUnion()) {
-      stat = computeStatFromUnionBlock(this);
-    } else {
-      stat = computeStatFromTasks();
-    }
-    TableMeta meta = writeStat(this, stat);
-    meta.setStat(stat);
-    setTableMeta(meta);
-    return meta;
-  }
-
-  private TableStat computeStatFromTasks() {
-    List<TableStat> stats = Lists.newArrayList();
-    for (QueryUnit unit : getQueryUnits()) {
-      stats.add(unit.getStats());
-    }
-    TableStat tableStat = StatisticsUtil.aggregateTableStat(stats);
-    return tableStat;
-  }
-
-  private TableMeta writeStat(SubQuery subQuery, TableStat stat)
-      throws IOException {
-    ExecutionBlock execBlock = subQuery.getBlock();
-    StoreTableNode storeTableNode = execBlock.getStoreTableNode();
-    TableMeta meta = toTableMeta(storeTableNode);
-    meta.setStat(stat);
-    sm.writeTableMeta(sm.getTablePath(execBlock.getOutputName()), meta);
-    return meta;
-  }
-
-  private static TableMeta toTableMeta(StoreTableNode store) {
-    if (store.hasOptions()) {
-      return CatalogUtil.newTableMeta(store.getOutSchema(),
-          store.getStorageType(), store.getOptions());
-    } else {
-      return CatalogUtil.newTableMeta(store.getOutSchema(),
-          store.getStorageType());
-    }
-  }
-
-  private void stopScheduler() {
-    // If there are launched TaskRunners, send the 'shouldDie' message to all r
-    // via received task requests.
-    if (taskScheduler != null) {
-      taskScheduler.stop();
-    }
-  }
-
-  private void releaseContainers() {
-    // If there are still live TaskRunners, try to kill the containers.
-    eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP ,getId(),
-        containers.values()));
-  }
-
-  private void finish() {
-    TableMeta meta = null;
-    try {
-      meta = buildTableMeta();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-    setTableMeta(meta);
-    setFinishTime();
-    eventHandler.handle(new SubQuerySucceeEvent(getId(), meta));
-  }
-
-  @Override
-  public void handle(SubQueryEvent event) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType());
-    }
-
-    try {
-      writeLock.lock();
-      SubQueryState oldState = getState();
-      try {
-        getStateMachine().doTransition(event.getType(), event);
-      } catch (InvalidStateTransitonException e) {
-        LOG.error("Can't handle this event at current state", e);
-        eventHandler.handle(new SubQueryEvent(getId(),
-            SubQueryEventType.SQ_INTERNAL_ERROR));
-      }
-
-      // notify the eventhandler of state change
-      if (LOG.isDebugEnabled()) {
-        if (oldState != getState()) {
-          LOG.debug(getId() + " SubQuery Transitioned from " + oldState + " to "
-              + getState());
-        }
-      }
-    }
-
-    finally {
-      writeLock.unlock();
-    }
-  }
-
-  private static class InitAndRequestContainer implements MultipleArcTransition<SubQuery,
-      SubQueryEvent, SubQueryState> {
-
-    @Override
-    public SubQueryState transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
-      subQuery.setStartTime();
-      ExecutionBlock execBlock = subQuery.getBlock();
-      SubQueryState state;
-
-      try {
-        // Union operator does not require actual query processing. It is performed logically.
-        if (execBlock.hasUnion()) {
-          subQuery.finish();
-          state = SubQueryState.SUCCEEDED;
-        } else {
-          setRepartitionIfNecessary(subQuery);
-          createTasks(subQuery);
-
-          if (subQuery.tasks.size() == 0) { // if there is no tasks
-            subQuery.finish();
-            return SubQueryState.SUCCEEDED;
-          } else {
-            initTaskScheduler(subQuery);
-            allocateContainers(subQuery);
-            return SubQueryState.INIT;
-          }
-        }
-      } catch (Exception e) {
-        LOG.warn("SubQuery (" + subQuery.getId() + ") failed", e);
-        subQuery.eventHandler.handle(
-            new QueryDiagnosticsUpdateEvent(subQuery.getId().getQueryId(), e.getMessage()));
-        subQuery.eventHandler.handle(
-            new SubQueryCompletedEvent(subQuery.getId(), SubQueryState.FAILED));
-        return SubQueryState.FAILED;
-      }
-
-      return state;
-    }
-
-    private void initTaskScheduler(SubQuery subQuery) {
-      subQuery.taskScheduler = new TaskSchedulerImpl(subQuery.context);
-      subQuery.taskScheduler.init(subQuery.context.getConf());
-      subQuery.taskScheduler.start();
-    }
-
-    /**
-     * If a parent block requires a repartition operation, the method sets proper repartition
-     * methods and the number of partitions to a given subquery.
-     */
-    private static void setRepartitionIfNecessary(SubQuery subQuery) {
-      if (subQuery.getBlock().hasParentBlock()) {
-        int numTasks = calculatePartitionNum(subQuery);
-        Repartitioner.setPartitionNumberForTwoPhase(subQuery, numTasks);
-      }
-    }
-
-    /**
-     * Getting the desire number of partitions according to the volume of input data.
-     * This method is only used to determine the partition key number of hash join or aggregation.
-     *
-     * @param subQuery
-     * @return
-     */
-    public static int calculatePartitionNum(SubQuery subQuery) {
-      TajoConf conf = subQuery.context.getConf();
-      ExecutionBlock parent = subQuery.getBlock().getParentBlock();
-
-      GroupbyNode grpNode = null;
-      if (parent != null) {
-        grpNode = (GroupbyNode) PlannerUtil.findTopNode(
-            parent.getPlan(), ExprType.GROUP_BY);
-      }
-
-      // Is this subquery the first step of join?
-      if (parent != null && parent.getScanNodes().length == 2) {
-        Iterator<ExecutionBlock> child = parent.getChildBlocks().iterator();
-
-        // for inner
-        ExecutionBlock outer = child.next();
-        long outerVolume = getInputVolume(subQuery.context, outer);
-
-        // for inner
-        ExecutionBlock inner = child.next();
-        long innerVolume = getInputVolume(subQuery.context, inner);
-        LOG.info("Outer volume: " + Math.ceil((double)outerVolume / 1048576));
-        LOG.info("Inner volume: " + Math.ceil((double)innerVolume / 1048576));
-
-        long smaller = Math.min(outerVolume, innerVolume);
-
-        int mb = (int) Math.ceil((double)smaller / 1048576);
-        LOG.info("Smaller Table's volume is approximately " + mb + " MB");
-        // determine the number of task
-        int taskNum = (int) Math.ceil((double)mb /
-            conf.getIntVar(ConfVars.JOIN_PARTITION_VOLUME));
-        LOG.info("The determined number of join partitions is " + taskNum);
-        return taskNum;
-
-        // Is this subquery the first step of group-by?
-      } else if (grpNode != null) {
-
-        if (grpNode.getGroupingColumns().length == 0) {
-          return 1;
-        } else {
-          long volume = getInputVolume(subQuery.context, subQuery.block);
-
-          int mb = (int) Math.ceil((double)volume / 1048576);
-          LOG.info("Table's volume is approximately " + mb + " MB");
-          // determine the number of task
-          int taskNum = (int) Math.ceil((double)mb /
-              conf.getIntVar(ConfVars.AGGREGATION_PARTITION_VOLUME));
-          LOG.info("The determined number of aggregation partitions is " + taskNum);
-          return taskNum;
-        }
-      } else {
-        LOG.info("============>>>>> Unexpected Case! <<<<<================");
-        long volume = getInputVolume(subQuery.context, subQuery.block);
-
-        int mb = (int) Math.ceil((double)volume / 1048576);
-        LOG.info("Table's volume is approximately " + mb + " MB");
-        // determine the number of task per 128MB
-        int taskNum = (int) Math.ceil((double)mb / 128);
-        LOG.info("The determined number of partitions is " + taskNum);
-        return taskNum;
-      }
-    }
-
-    private static void createTasks(SubQuery subQuery) throws IOException {
-      ExecutionBlock execBlock = subQuery.getBlock();
-      QueryUnit [] tasks;
-      if (execBlock.isLeafBlock() && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
-        tasks = createLeafTasks(subQuery);
-
-      } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
-        tasks = Repartitioner.createJoinTasks(subQuery);
-
-      } else { // Case 3: Others (Sort or Aggregation)
-        int numTasks = getNonLeafTaskNum(subQuery);
-        SubQueryId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
-        SubQuery child = subQuery.context.getSubQuery(childId);
-        tasks = Repartitioner.createNonLeafTask(subQuery, child, numTasks);
-      }
-
-      LOG.info("Create " + tasks.length + " Tasks");
-
-      for (QueryUnit task : tasks) {
-        subQuery.addTask(task);
-      }
-    }
-
-    /**
-     * Getting the desire number of tasks according to the volume of input data
-     *
-     * @param subQuery
-     * @return
-     */
-    public static int getNonLeafTaskNum(SubQuery subQuery) {
-      // Getting intermediate data size
-      long volume = getInputVolume(subQuery.context, subQuery.getBlock());
-
-      int mb = (int) Math.ceil((double)volume / 1048576);
-      LOG.info("Table's volume is approximately " + mb + " MB");
-      // determine the number of task per 64MB
-      int maxTaskNum = (int) Math.ceil((double)mb / 64);
-      LOG.info("The determined number of non-leaf tasks is " + maxTaskNum);
-      return maxTaskNum;
-    }
-
-    public static long getInputVolume(QueryContext context, ExecutionBlock execBlock) {
-      CatalogService catalog = context.getCatalog();
-      if (execBlock.isLeafBlock()) {
-        ScanNode outerScan = execBlock.getScanNodes()[0];
-        TableStat stat = catalog.getTableDesc(outerScan.getTableId()).getMeta().getStat();
-        return stat.getNumBytes();
-      } else {
-        long aggregatedVolume = 0;
-        for (ExecutionBlock childBlock : execBlock.getChildBlocks()) {
-          SubQuery subquery = context.getSubQuery(childBlock.getId());
-          aggregatedVolume += subquery.getTableStat().getNumBytes();
-        }
-
-        return aggregatedVolume;
-      }
-    }
-
-    public static void allocateContainers(SubQuery subQuery) {
-      ExecutionBlock execBlock = subQuery.getBlock();
-      QueryUnit [] tasks = subQuery.getQueryUnits();
-
-      int numClusterNodes = subQuery.getContext().getNumClusterNode();
-      int numRequest = Math.min(tasks.length, numClusterNodes * 4);
-
-      final Resource resource = Records.newRecord(Resource.class);
-      if (tasks.length <= numClusterNodes) {
-        resource.setMemory(subQuery.context.getMaxContainerCapability());
-      } else {
-        resource.setMemory(2000);
-      }
-
-      Priority priority = Records.newRecord(Priority.class);
-      priority.setPriority(subQuery.getPriority());
-      ContainerAllocationEvent event =
-          new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ,
-              subQuery.getId(), priority, resource, numRequest,
-              execBlock.isLeafBlock(), 0.0f);
-      subQuery.eventHandler.handle(event);
-    }
-
-    private static QueryUnit [] createLeafTasks(SubQuery subQuery) throws IOException {
-      ExecutionBlock execBlock = subQuery.getBlock();
-      ScanNode[] scans = execBlock.getScanNodes();
-      Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");
-      TableMeta meta;
-      Path inputPath;
-
-      ScanNode scan = scans[0];
-      TableDesc desc = subQuery.context.getCatalog().getTableDesc(scan.getTableId());
-      inputPath = desc.getPath();
-      meta = desc.getMeta();
-
-      // TODO - should be change the inner directory
-      Path oldPath = new Path(inputPath, "data");
-      FileSystem fs = inputPath.getFileSystem(subQuery.context.getConf());
-      if (fs.exists(oldPath)) {
-        inputPath = oldPath;
-      }
-      List<Fragment> fragments = subQuery.getStorageManager().getSplits(scan.getTableId(), meta, inputPath);
-
-      QueryUnit queryUnit;
-      List<QueryUnit> queryUnits = new ArrayList<QueryUnit>();
-
-      int i = 0;
-      for (Fragment fragment : fragments) {
-        queryUnit = newQueryUnit(subQuery, i++, fragment);
-        queryUnits.add(queryUnit);
-      }
-
-      return queryUnits.toArray(new QueryUnit[queryUnits.size()]);
-    }
-
-    private static QueryUnit newQueryUnit(SubQuery subQuery, int taskId, Fragment fragment) {
-      ExecutionBlock execBlock = subQuery.getBlock();
-      QueryUnit unit = new QueryUnit(
-          QueryIdFactory.newQueryUnitId(subQuery.getId(), taskId), execBlock.isLeafBlock(),
-          subQuery.eventHandler);
-      unit.setLogicalPlan(execBlock.getPlan());
-      unit.setFragment2(fragment);
-      return unit;
-    }
-  }
-
-  int i = 0;
-  private static class ContainerLaunchTransition
-      implements SingleArcTransition<SubQuery, SubQueryEvent> {
-
-    @Override
-    public void transition(SubQuery subQuery, SubQueryEvent event) {
-      SubQueryContainerAllocationEvent allocationEvent =
-          (SubQueryContainerAllocationEvent) event;
-      for (Container container : allocationEvent.getAllocatedContainer()) {
-        ContainerId cId = container.getId();
-        if (subQuery.containers.containsKey(cId)) {
-          LOG.info(">>>>>>>>>>>> Duplicate Container! <<<<<<<<<<<");
-        }
-        subQuery.containers.put(cId, container);
-        // TODO - This is debugging message. Should be removed
-        subQuery.i++;
-      }
-      LOG.info("SubQuery (" + subQuery.getId() + ") has " + subQuery.i + " containers!");
-      subQuery.eventHandler.handle(
-          new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
-              subQuery.getId(), allocationEvent.getAllocatedContainer()));
-
-      subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
-          SubQueryEventType.SQ_START));
-    }
-  }
-
-  private static class StartTransition implements
-      MultipleArcTransition<SubQuery, SubQueryEvent, SubQueryState> {
-
-    @Override
-    public SubQueryState transition(SubQuery subQuery,
-                           SubQueryEvent subQueryEvent) {
-      // schedule tasks
-      try {
-        for (QueryUnitId taskId : subQuery.tasks.keySet()) {
-          subQuery.eventHandler.handle(new TaskEvent(taskId, TaskEventType.T_SCHEDULE));
-        }
-
-        return  SubQueryState.RUNNING;
-      } catch (Exception e) {
-        LOG.warn("SubQuery (" + subQuery.getId() + ") failed", e);
-        return SubQueryState.FAILED;
-      }
-    }
-  }
-
-  private static class TaskCompletedTransition
-      implements SingleArcTransition<SubQuery, SubQueryEvent> {
-
-    @Override
-    public void transition(SubQuery subQuery,
-                                     SubQueryEvent event) {
-      subQuery.completedTaskCount++;
-      SubQueryTaskEvent taskEvent = (SubQueryTaskEvent) event;
-      QueryUnitAttempt task = subQuery.getQueryUnit(taskEvent.getTaskId()).getSuccessfulAttempt();
-
-      LOG.info(subQuery.getId() + " SubQuery Succeeded " + subQuery.completedTaskCount + "/"
-          + subQuery.tasks.size() + " on " + task.getHost());
-      if (subQuery.completedTaskCount == subQuery.tasks.size()) {
-        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
-            SubQueryEventType.SQ_SUBQUERY_COMPLETED));
-      }
-    }
-  }
-
-  private static class SubQueryCompleteTransition
-      implements SingleArcTransition<SubQuery, SubQueryEvent> {
-
-    @Override
-    public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
-      // TODO - Commit subQuery & do cleanup
-      // TODO - records succeeded, failed, killed completed task
-      // TODO - records metrics
-      subQuery.stopScheduler();
-      subQuery.releaseContainers();
-      subQuery.finish();
-    }
-  }
-
-  private static class InternalErrorTransition
-      implements SingleArcTransition<SubQuery, SubQueryEvent> {
-
-    @Override
-    public void transition(SubQuery subQuery,
-                           SubQueryEvent subQueryEvent) {
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQueryState.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQueryState.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQueryState.java
deleted file mode 100644
index 525daea..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/SubQueryState.java
+++ /dev/null
@@ -1,28 +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;
-
-public enum SubQueryState {
-  NEW,
-  CONTAINER_ALLOCATED,
-  INIT,
-  RUNNING,
-  SUCCEEDED,
-  FAILED
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 5f379ab..b84b51b 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
@@ -44,12 +44,12 @@ import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.conf.TajoConf;
 import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskStatusProto;
 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.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryMasterManager;
+import org.apache.tajo.master.querymaster.QueryMasterManagerService;
 import org.apache.tajo.storage.StorageManager;
 import org.apache.tajo.webapp.StaticHttpServer;
 
@@ -78,7 +78,8 @@ public class TajoMaster extends CompositeService {
   private StorageManager storeManager;
   private GlobalEngine globalEngine;
   private AsyncDispatcher dispatcher;
-  private ClientService clientService;
+  private TajoMasterClientService tajoMasterClientService;
+  private QueryMasterManagerService queryMasterManagerService;
   private YarnRPC yarnRPC;
 
   //Web Server
@@ -97,6 +98,8 @@ public class TajoMaster extends CompositeService {
 
 
     try {
+      RackResolver.init(conf);
+
       webServer = StaticHttpServer.getInstance(this ,"admin", null, 8080 ,
           true, null, context.getConf(), null);
       webServer.start();
@@ -110,7 +113,6 @@ public class TajoMaster extends CompositeService {
       this.defaultFS = basePath.getFileSystem(conf);
       conf.set("fs.defaultFS", defaultFS.getUri().toString());
       LOG.info("FileSystem (" + this.defaultFS.getUri() + ") is initialized.");
-
       if (!defaultFS.exists(basePath)) {
         defaultFS.mkdirs(basePath);
         LOG.info("Tajo Base dir (" + basePath + ") is created.");
@@ -147,12 +149,11 @@ public class TajoMaster extends CompositeService {
       globalEngine = new GlobalEngine(context);
       addIfService(globalEngine);
 
-      dispatcher.register(QueryEventType.class, new QueryEventDispatcher());
-
-      clientService = new ClientService(context);
-      addIfService(clientService);
+      tajoMasterClientService = new TajoMasterClientService(context);
+      addIfService(tajoMasterClientService);
 
-      RackResolver.init(conf);
+      queryMasterManagerService = new QueryMasterManagerService(context);
+      addIfService(queryMasterManagerService);
     } catch (Exception e) {
        e.printStackTrace();
     }
@@ -281,6 +282,10 @@ public class TajoMaster extends CompositeService {
       LOG.error(e);
     }
 
+    for(QueryMasterManager eachQuery: getContext().getAllQueries().values()) {
+      eachQuery.stop();
+    }
+
     super.stop();
     LOG.info("TajoMaster main thread exiting");
   }
@@ -306,18 +311,18 @@ public class TajoMaster extends CompositeService {
   }
 
   // TODO - to be improved
-  public Collection<TaskStatusProto> getProgressQueries() {
+  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);
-    }
-  }
+//  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);
@@ -331,13 +336,14 @@ public class TajoMaster extends CompositeService {
       master.init(conf);
       master.start();
     } catch (Throwable t) {
-      LOG.fatal("Error starting JobHistoryServer", 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, QueryMaster> queries = Maps.newConcurrentMap();
+    private final Map<QueryId, QueryMasterManager> queries = Maps.newConcurrentMap();
     private final TajoConf conf;
 
     public MasterContext(TajoConf conf) {
@@ -352,14 +358,18 @@ public class TajoMaster extends CompositeService {
       return clock;
     }
 
-    public QueryMaster getQuery(QueryId queryId) {
+    public QueryMasterManager getQuery(QueryId queryId) {
       return queries.get(queryId);
     }
 
-    public Map<QueryId, QueryMaster> getAllQueries() {
+    public Map<QueryId, QueryMasterManager> getAllQueries() {
       return queries;
     }
 
+    public void addQuery(QueryId queryId, QueryMasterManager queryMasterManager) {
+      queries.put(queryId, queryMasterManager);
+    }
+
     public AsyncDispatcher getDispatcher() {
       return dispatcher;
     }
@@ -384,8 +394,12 @@ public class TajoMaster extends CompositeService {
       return yarnRPC;
     }
 
-    public ClientService getClientService() {
-      return clientService;
+    public TajoMasterClientService getClientService() {
+      return tajoMasterClientService;
+    }
+
+    public QueryMasterManagerService getQueryMasterManagerService() {
+      return queryMasterManagerService;
     }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..273a3c1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -0,0 +1,412 @@
+/**
+ * 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.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+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.net.NetUtils;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.exception.AlreadyExistsTableException;
+import org.apache.tajo.catalog.exception.NoSuchTableException;
+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.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.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.TajoIdUtils;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+
+public class TajoMasterClientService extends AbstractService {
+  private final static Log LOG = LogFactory.getLog(TajoMasterClientService.class);
+  private final MasterContext context;
+  private final TajoConf conf;
+  private final CatalogService catalog;
+  private final TajoMasterClientProtocolServiceHandler clientHandler;
+  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 TajoMasterClientService(MasterContext context) {
+    super(TajoMasterClientService.class.getName());
+    this.context = context;
+    this.conf = context.getConf();
+    this.catalog = context.getCatalog();
+    this.clientHandler = new TajoMasterClientProtocolServiceHandler();
+  }
+
+  @Override
+  public void start() {
+
+    // start the rpc server
+    String confClientServiceAddr = conf.getVar(ConfVars.CLIENT_SERVICE_ADDRESS);
+    InetSocketAddress initIsa = NetUtils.createSocketAddr(confClientServiceAddr);
+    try {
+      server = new ProtoBlockingRpcServer(TajoMasterClientProtocol.class, clientHandler, initIsa);
+    } catch (Exception e) {
+      LOG.error(e);
+    }
+    server.start();
+    bindAddress = server.getBindAddress();
+    this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS,
+        org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
+    LOG.info("Instantiated TajoMasterClientService at " + this.bindAddress);
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    if (server != null) {
+      server.shutdown();
+    }
+    super.stop();
+  }
+
+  public InetSocketAddress getBindAddress() {
+    return this.bindAddress;
+  }
+
+  public int getHttpPort() {
+    return 0;
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // TajoMasterClientProtocolService
+  /////////////////////////////////////////////////////////////////////////////
+  public class TajoMasterClientProtocolServiceHandler implements TajoMasterClientProtocolService.BlockingInterface {
+    @Override
+    public BoolProto updateSessionVariables(RpcController controller,
+                                            UpdateSessionVariableRequest request)
+        throws ServiceException {
+      return null;
+    }
+
+    @Override
+    public SubmitQueryResponse 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.error(msg, e);
+        } else {
+          LOG.error(msg);
+        }
+        build.setErrorMessage(msg);
+        return build.build();
+      }
+
+      LOG.info("Query " + queryId + " is submitted");
+      build.setResultCode(ResultCode.OK);
+      build.setQueryId(queryId.getProto());
+
+      return build.build();
+    }
+
+    @Override
+    public UpdateQueryResponse updateQuery(RpcController controller,
+                                           QueryRequest request)
+        throws ServiceException {
+
+      UpdateQueryResponse.Builder builder = UpdateQueryResponse.newBuilder();
+      try {
+        context.getGlobalEngine().updateQuery(request.getQuery());
+        builder.setResultCode(ResultCode.OK);
+        return builder.build();
+      } catch (Exception e) {
+        builder.setResultCode(ResultCode.ERROR);
+        if (e.getMessage() == null) {
+          builder.setErrorMessage(ExceptionUtils.getStackTrace(e));
+        }
+        return builder.build();
+      }
+    }
+
+    @Override
+    public GetQueryResultResponse getQueryResult(RpcController controller,
+                                                 GetQueryResultRequest request)
+        throws ServiceException {
+      QueryId queryId = new QueryId(request.getQueryId());
+      QueryMasterManager queryMasterManager = context.getQuery(queryId);
+
+      GetQueryResultResponse.Builder builder
+          = GetQueryResultResponse.newBuilder();
+      switch (queryMasterManager.getState()) {
+        case QUERY_SUCCEEDED:
+          builder.setTableDesc((TableDescProto) queryMasterManager.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 GetQueryListResponse getQueryList(RpcController controller,
+                                             GetQueryListRequest request)
+        throws ServiceException {
+      return null;
+    }
+
+    @Override
+    public GetQueryStatusResponse getQueryStatus(RpcController controller,
+                                                 GetQueryStatusRequest request)
+        throws ServiceException {
+
+      GetQueryStatusResponse.Builder builder
+          = GetQueryStatusResponse.newBuilder();
+      QueryId queryId = new QueryId(request.getQueryId());
+      builder.setQueryId(request.getQueryId());
+
+      if (queryId.equals(TajoIdUtils.NullQueryId)) {
+        builder.setResultCode(ResultCode.OK);
+        builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
+      } else {
+        QueryMasterManager queryMasterManager = context.getQuery(queryId);
+        if (queryMasterManager != null) {
+          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());
+          }
+
+          if (queryMasterManager.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+            builder.setFinishTime(queryMasterManager.getFinishTime());
+          } else {
+            builder.setFinishTime(System.currentTimeMillis());
+          }
+        } else {
+          builder.setResultCode(ResultCode.ERROR);
+          builder.setErrorMessage("No such query: " + queryId.toString());
+        }
+      }
+
+      return builder.build();
+    }
+
+    @Override
+    public BoolProto killQuery(RpcController controller,
+                               ApplicationAttemptIdProto request)
+        throws ServiceException {
+      QueryId queryId = new QueryId(request);
+      QueryMasterManager queryMasterManager = context.getQuery(queryId);
+      //queryMasterManager.handle(new QueryEvent(queryId, QueryEventType.KILL));
+
+      return BOOL_TRUE;
+    }
+
+    @Override
+    public GetClusterInfoResponse getClusterInfo(RpcController controller,
+                                                 GetClusterInfoRequest request)
+        throws ServiceException {
+      return null;
+    }
+
+    @Override
+    public BoolProto existTable(RpcController controller,
+                                StringProto tableNameProto)
+        throws ServiceException {
+      String tableName = tableNameProto.getValue();
+      if (catalog.existsTable(tableName)) {
+        return BOOL_TRUE;
+      } else {
+        return BOOL_FALSE;
+      }
+    }
+
+    @Override
+    public GetTableListResponse getTableList(RpcController controller,
+                                             GetTableListRequest request)
+        throws ServiceException {
+      Collection<String> tableNames = catalog.getAllTableNames();
+      GetTableListResponse.Builder builder = GetTableListResponse.newBuilder();
+      builder.addAllTables(tableNames);
+      return builder.build();
+    }
+
+    @Override
+    public TableResponse getTableDesc(RpcController controller,
+                                      GetTableDescRequest request)
+        throws ServiceException {
+      String name = request.getTableName();
+      if (catalog.existsTable(name)) {
+        return TableResponse.newBuilder()
+            .setTableDesc((TableDescProto) catalog.getTableDesc(name).getProto())
+            .build();
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public TableResponse createTable(RpcController controller, CreateTableRequest request)
+        throws ServiceException {
+      Path path = new Path(request.getPath());
+      TableMeta meta = new TableMetaImpl(request.getMeta());
+      TableDesc desc;
+      try {
+        desc = context.getGlobalEngine().createTable(request.getName(), meta, path);
+      } catch (Exception e) {
+        return TableResponse.newBuilder().setErrorMessage(e.getMessage()).build();
+      }
+
+      return TableResponse.newBuilder().setTableDesc((TableDescProto) desc.getProto()).build();
+    }
+
+    @Override
+    public BoolProto dropTable(RpcController controller,
+                               StringProto tableNameProto)
+        throws ServiceException {
+      context.getGlobalEngine().dropTable(tableNameProto.getValue());
+      return BOOL_TRUE;
+    }
+
+    @Override
+    public TableResponse attachTable(RpcController controller,
+                                     AttachTableRequest request)
+        throws ServiceException {
+
+      TableDesc desc;
+      if (catalog.existsTable(request.getName())) {
+        throw new AlreadyExistsTableException(request.getName());
+      }
+
+      Path path = new Path(request.getPath());
+
+      LOG.info(path.toUri());
+
+      TableMeta meta;
+      try {
+        meta = TableUtil.getTableMeta(conf, path);
+      } catch (IOException e) {
+        throw new RemoteException(e);
+      }
+
+      FileSystem fs;
+
+      // for legacy table structure
+      Path tablePath = new Path(path, "data");
+      try {
+        fs = path.getFileSystem(conf);
+        if (!fs.exists(tablePath)) {
+          tablePath = path;
+        }
+      } catch (IOException e) {
+        LOG.error(e);
+        return null;
+      }
+
+      if (meta.getStat() == null) {
+        long totalSize = 0;
+        try {
+          totalSize = calculateSize(tablePath);
+        } catch (IOException e) {
+          LOG.error("Cannot calculate the size of the relation", e);
+          return null;
+        }
+
+        meta = new TableMetaImpl(meta.getProto());
+        TableStat stat = new TableStat();
+        stat.setNumBytes(totalSize);
+        meta.setStat(stat);
+      }
+
+      desc = new TableDescImpl(request.getName(), meta, path);
+      catalog.addTable(desc);
+      LOG.info("Table " + desc.getId() + " is attached ("
+          + meta.getStat().getNumBytes() + ")");
+
+      return TableResponse.newBuilder().
+          setTableDesc((TableDescProto) desc.getProto())
+          .build();
+    }
+
+    @Override
+    public BoolProto detachTable(RpcController controller,
+                                 StringProto tableNameProto)
+        throws ServiceException {
+      String tableName = tableNameProto.getValue();
+      if (!catalog.existsTable(tableName)) {
+        throw new NoSuchTableException(tableName);
+      }
+
+      catalog.deleteTable(tableName);
+
+      LOG.info("Table " + tableName + " is detached");
+      return BOOL_TRUE;
+    }
+  }
+
+  private long calculateSize(Path path) throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    long totalSize = 0;
+    for (FileStatus status : fs.listStatus(path)) {
+      totalSize += status.getLen();
+    }
+
+    return totalSize;
+  }
+}


[5/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
index 8e0abd0..d8ddb46 100644
--- 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
@@ -21,76 +21,45 @@ 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.ApplicationConstants.Environment;
-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.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.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.service.AbstractService;
-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.SubQueryId;
 import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.QueryMaster.QueryContext;
 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.pullserver.PullServerAuxService;
+import org.apache.tajo.master.querymaster.QueryMaster;
+import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
+import org.apache.tajo.worker.TaskRunner;
 
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.util.*;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Vector;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 public class TaskRunnerLauncherImpl extends AbstractService implements TaskRunnerLauncher {
 
   /** Class Logger */
   private static final Log LOG = LogFactory.getLog(TaskRunnerLauncherImpl.class);
-  private final YarnRPC yarnRPC;
-  private final static RecordFactory recordFactory =
-      RecordFactoryProvider.getRecordFactory(null);
   private QueryContext context;
-  private final String taskListenerHost;
-  private final int taskListenerPort;
+  private final String queryMasterHost;
+  private final int queryMasterPort;
 
   // For ContainerLauncherSpec
-  private static AtomicBoolean initialClasspathFlag = new AtomicBoolean();
-  private static String initialClasspath = null;
-  private static final Object classpathLock = new Object();
-  private Object commonContainerSpecLock = 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;
 
   public TaskRunnerLauncherImpl(QueryContext context) {
     super(TaskRunnerLauncherImpl.class.getName());
     this.context = context;
-    taskListenerHost = context.getTaskListener().getHostName();
-    taskListenerPort = context.getTaskListener().getPort();
-    yarnRPC = context.getYarnRPC();
+    queryMasterHost = context.getQueryMasterServiceAddress().getHostName();
+    queryMasterPort = context.getQueryMasterServiceAddress().getPort();
     executorService = Executors.newFixedThreadPool(
         context.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
   }
@@ -100,7 +69,22 @@ public class TaskRunnerLauncherImpl extends AbstractService implements TaskRunne
   }
 
   public void stop() {
-    executorService.shutdown();
+    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.getContainers();
+    for(ContainerProxy eachProxy: containers.values()) {
+      try {
+        eachProxy.kill();
+      } catch (Exception e) {
+      }
+    }
     super.stop();
   }
 
@@ -114,427 +98,82 @@ public class TaskRunnerLauncherImpl extends AbstractService implements TaskRunne
   }
 
   private void launchTaskRunners(SubQueryId subQueryId, Collection<Container> containers) {
+    commonContainerSpec = ContainerProxy.createCommonContainerLaunchContext(getConfig());
     for (Container container : containers) {
-      final ContainerProxy proxy = new ContainerProxy(container, subQueryId);
-      executorService.submit(new LaunchRunner(proxy));
+      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;
-    public LaunchRunner(ContainerProxy proxy) {
+    private final ContainerId id;
+
+    public LaunchRunner(ContainerId id, ContainerProxy proxy) {
       this.proxy = proxy;
+      this.id = id;
     }
     @Override
     public void run() {
-      proxy.launch();
+      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(proxy));
+      executorService.submit(new KillRunner(container.getId(), proxy));
     }
   }
 
   private class KillRunner implements Runnable {
     private final ContainerProxy proxy;
-    public KillRunner(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);
     }
   }
 
-
-  /**
-   * 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;
-    }
-  }
-
-  private ContainerLaunchContext createCommonContainerLaunchContext() {
-    TajoConf conf = (TajoConf) getConfig();
-
-    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(Environment.SHELL.name(), "/bin/bash");
-    environment.put(Environment.JAVA_HOME.name(), System.getenv(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")) {
-      environment.put(Environment.CLASSPATH.name(), prop.getProperty(
-          "java.class.path", null));
-    } 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:./*");
-      environment.put("HADOOP_HOME", System.getenv("HADOOP_HOME"));
-      environment.put(
-          Environment.HADOOP_COMMON_HOME.name(),
-          System.getenv("HADOOP_HOME"));
-      environment.put(
-          Environment.HADOOP_HDFS_HOME.name(),
-          System.getenv("HADOOP_HOME"));
-      environment.put(
-          Environment.HADOOP_YARN_HOME.name(),
-          System.getenv("HADOOP_HOME"));
-      environment.put("TAJO_BASE_CLASSPATH", System.getenv("TAJO_BASE_CLASSPATH"));
-      environment.put(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.default.name"));
-    LOG.info("defaultFS: " + conf.get("fs.defaultFS"));
-    try {
-      fs = FileSystem.get(conf);
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-    FileContext fsCtx = null;
-    try {
-      fsCtx = FileContext.getFileContext(getConfig());
-    } catch (UnsupportedFileSystemException e) {
-      e.printStackTrace();
-    }
-
-    LOG.info("Writing a QueryConf to HDFS and add to local environment");
-    Path queryConfPath = new Path(fs.getHomeDirectory(), QueryConf.FILENAME);
-    try {
-      writeConf(conf, queryConfPath);
-
-      LocalResource queryConfSrc = createApplicationResource(fsCtx,
-          queryConfPath, LocalResourceType.FILE);
-      localResources.put(QueryConf.FILENAME,  queryConfSrc);
-
-      ctx.setLocalResources(localResources);
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-    // 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;
-  }
-
-  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, getConfig());
-          }
-        });
-    return proxy;
-  }
-
-  private 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 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();
-    }
-  }
-
-  private static enum ContainerState {
-    PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
-  }
-
-  public class ContainerProxy {
-    private 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;
+  public class TaskRunnerContainerProxy extends ContainerProxy {
     private final SubQueryId subQueryId;
 
-    public ContainerProxy(Container container, SubQueryId subQueryId) {
-      this.state = ContainerState.PREP;
-      this.container = container;
-      this.containerID = container.getId();
-      NodeId nodeId = container.getNodeId();
-      this.containerMgrAddress = nodeId.getHost() + ":" + nodeId.getPort();;
-      this.containerToken = container.getContainerToken();
+    public TaskRunnerContainerProxy(QueryMaster.QueryContext context, Configuration conf, YarnRPC yarnRPC,
+                                    Container container, SubQueryId subQueryId) {
+      super(context, conf, yarnRPC, container);
       this.subQueryId = subQueryId;
     }
 
-    public synchronized boolean isCompletelyDone() {
-      return state == ContainerState.DONE || state == ContainerState.FAILED;
-    }
-
-    @SuppressWarnings("unchecked")
-    public synchronized void launch() {
-      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();
-
-        // 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);
-        }
-
-        // after launching, send launched event to task attempt to move
-        // it from ASSIGNED to RUNNING state
-//      context.getEventHandler().handle(new AMContainerEventLaunched(containerID, port));
-
-        // this is workaround code
-        context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
-
-        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, getConfig());
-        }
-      }
+    @Override
+    protected void containerStarted() {
+      context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
     }
 
-    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);
-//          context.getEventHandler().handle(
-//              new AMContainerEventStopFailed(containerID, message));
-          LOG.warn(message);
-          this.state = ContainerState.DONE;
-          return;
-        } finally {
-          if (proxy != null) {
-            yarnRPC.stopProxy(proxy, getConfig());
-          }
-        }
-        this.state = ContainerState.DONE;
-      }
+    @Override
+    protected String getId() {
+      return subQueryId.toString();
     }
 
-    public ContainerLaunchContext createContainerLaunchContext() {
-      synchronized (commonContainerSpecLock) {
-        if (commonContainerSpec == null) {
-          commonContainerSpec = createCommonContainerLaunchContext();
-        }
-      }
-
-      // Setup environment by cloning from common env.
-      Map<String, String> env = commonContainerSpec.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 : commonContainerSpec
-          .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("org.apache.tajo.worker.TaskRunner");
-      vargs.add(taskListenerHost); // tasklistener hostname
-      vargs.add(String.valueOf(taskListenerPort)); // tasklistener hostname
-      vargs.add(subQueryId.toString()); // subqueryId
-      vargs.add(containerMgrAddress); // nodeId
-      vargs.add(containerID.toString()); // containerId
-
-      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, commonContainerSpec.getUser(),
-          container.getResource(), commonContainerSpec.getLocalResources(), myEnv, commands,
-          myServiceData, null, new HashMap<ApplicationAccessType, String>());
+    @Override
+    protected String getRunnerClass() {
+      return TaskRunner.class.getCanonicalName();
     }
 
-    public String getHostName() {
-      return this.hostName;
-    }
+    @Override
+    protected Vector<CharSequence> getTaskParams() {
+      Vector<CharSequence> taskParams = new Vector<CharSequence>();
+      taskParams.add(queryMasterHost); // queryMaster hostname
+      taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
 
-    public int getPullServerPort() {
-      return this.port;
+      return taskParams;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerListener.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerListener.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerListener.java
deleted file mode 100644
index ce16897..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerListener.java
+++ /dev/null
@@ -1,172 +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 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.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.TajoIdProtos.QueryUnitAttemptIdProto;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.MasterWorkerProtos.QueryUnitRequestProto;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskCompletionReport;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskFatalErrorReport;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskStatusProto;
-import org.apache.tajo.ipc.MasterWorkerProtocol;
-import org.apache.tajo.ipc.MasterWorkerProtocol.MasterWorkerProtocolService;
-import org.apache.tajo.master.QueryMaster.QueryContext;
-import org.apache.tajo.master.event.TaskAttemptStatusUpdateEvent;
-import org.apache.tajo.master.event.TaskCompletionEvent;
-import org.apache.tajo.master.event.TaskFatalErrorEvent;
-import org.apache.tajo.master.event.TaskRequestEvent;
-import org.apache.tajo.rpc.ProtoAsyncRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-
-public class TaskRunnerListener extends AbstractService
-    implements MasterWorkerProtocolService.Interface {
-  
-  private final static Log LOG = LogFactory.getLog(
-      org.apache.tajo.master.cluster.WorkerListener.class);
-  private QueryContext context;
-  private ProtoAsyncRpcServer rpcServer;
-  private InetSocketAddress bindAddr;
-  private String addr;
-  
-  public TaskRunnerListener(final QueryContext context) throws Exception {
-    super(org.apache.tajo.master.cluster.WorkerListener.class.getName());
-    this.context = context;
-
-
-    InetSocketAddress initIsa =
-        new InetSocketAddress(InetAddress.getLocalHost(), 0);
-    if (initIsa.getAddress() == null) {
-      throw new IllegalArgumentException("Failed resolve of " + initIsa);
-    }
-    try {
-      this.rpcServer = new ProtoAsyncRpcServer(MasterWorkerProtocol.class,
-          this, initIsa);
-    } catch (Exception e) {
-      LOG.error(e);
-    }
-    this.rpcServer.start();
-    this.bindAddr = rpcServer.getBindAddress();
-    this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
-  }
-
-  @Override
-  public void init(Configuration conf) {
-    // 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(MasterWorkerProtocol.class,
-          this, initIsa);
-
-      this.rpcServer.start();
-      this.bindAddr = rpcServer.getBindAddress();
-      this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
-
-    } catch (Exception e) {
-      LOG.error(e);
-    }
-
-    // Get the master address
-    LOG.info(org.apache.tajo.master.cluster.WorkerListener.class.getSimpleName() + " is bind to " + addr);
-    context.getConf().setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
-
-    super.init(conf);
-  }
-
-  @Override
-  public void start() {
-
-
-    super.start();
-  }
-
-  @Override
-  public void stop() {
-    rpcServer.shutdown();
-    super.stop();
-  }
-  
-  public InetSocketAddress getBindAddress() {
-    return this.bindAddr;
-  }
-  
-  public String getAddress() {
-    return this.addr;
-  }
-
-  static BoolProto TRUE_PROTO = BoolProto.newBuilder().setValue(true).build();
-
-  @Override
-  public void getTask(RpcController controller, ContainerIdProto request,
-                      RpcCallback<QueryUnitRequestProto> done) {
-    context.getEventHandler().handle(new TaskRequestEvent(
-        new ContainerIdPBImpl(request), done));
-  }
-
-  @Override
-  public void statusUpdate(RpcController controller, TaskStatusProto request,
-                           RpcCallback<BoolProto> done) {
-    QueryUnitAttemptId attemptId = new QueryUnitAttemptId(request.getId());
-    context.getEventHandler().handle(new TaskAttemptStatusUpdateEvent(attemptId,
-        request));
-    done.run(TRUE_PROTO);
-  }
-
-  @Override
-  public void ping(RpcController controller,
-                   QueryUnitAttemptIdProto attemptIdProto,
-                   RpcCallback<BoolProto> done) {
-    // TODO - to be completed
-    QueryUnitAttemptId attemptId = new QueryUnitAttemptId(attemptIdProto);
-//    context.getQuery(attemptId.getQueryId()).getSubQuery(attemptId.getSubQueryId()).
-//        getQueryUnit(attemptId.getQueryUnitId()).getAttempt(attemptId).
-//        resetExpireTime();
-    done.run(TRUE_PROTO);
-  }
-
-  @Override
-  public void fatalError(RpcController controller, TaskFatalErrorReport report,
-                         RpcCallback<BoolProto> done) {
-    context.getEventHandler().handle(new TaskFatalErrorEvent(report));
-    done.run(TRUE_PROTO);
-  }
-
-  @Override
-  public void done(RpcController controller, TaskCompletionReport report,
-                       RpcCallback<BoolProto> done) {
-    context.getEventHandler().handle(new TaskCompletionEvent(report));
-    done.run(TRUE_PROTO);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 381c333..62e702d 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
@@ -29,18 +29,18 @@ import org.apache.hadoop.yarn.util.RackResolver;
 import org.apache.tajo.QueryIdFactory;
 import org.apache.tajo.QueryUnitAttemptId;
 import org.apache.tajo.SubQueryId;
-import org.apache.tajo.engine.MasterWorkerProtos;
 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.protocolrecords.QueryUnitRequest;
-import org.apache.tajo.master.QueryMaster.QueryContext;
-import org.apache.tajo.master.TaskRunnerLauncherImpl.ContainerProxy;
+import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
 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.QueryUnit;
 import org.apache.tajo.storage.Fragment;
 import org.apache.tajo.util.TajoIdUtils;
 
@@ -96,9 +96,11 @@ public class TaskSchedulerImpl extends AbstractService
             event = eventQueue.take();
             handleEvent(event);
           } catch (InterruptedException e) {
-            LOG.error("Returning, iterrupted : " + e);
+            //LOG.error("Returning, iterrupted : " + e);
+            break;
           }
         }
+        LOG.info("TaskScheduler eventHandlingThread stopped");
       }
     };
 
@@ -111,11 +113,12 @@ public class TaskSchedulerImpl extends AbstractService
           try {
             Thread.sleep(1000);
           } catch (InterruptedException e) {
-            LOG.warn(e);
+            break;
           }
 
           schedule();
         }
+        LOG.info("TaskScheduler schedulingThread stopped");
       }
     };
 
@@ -124,14 +127,13 @@ public class TaskSchedulerImpl extends AbstractService
   }
 
   private static final QueryUnitAttemptId NULL_ID;
-  private static final MasterWorkerProtos.QueryUnitRequestProto stopTaskRunnerReq;
+  private static final QueryMasterProtocol.QueryUnitRequestProto stopTaskRunnerReq;
   static {
     SubQueryId nullSubQuery =
         QueryIdFactory.newSubQueryId(TajoIdUtils.NullQueryId);
     NULL_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
 
-    MasterWorkerProtos.QueryUnitRequestProto.Builder builder =
-                MasterWorkerProtos.QueryUnitRequestProto.newBuilder();
+    QueryMasterProtocol.QueryUnitRequestProto.Builder builder = QueryMasterProtocol.QueryUnitRequestProto.newBuilder();
     builder.setId(NULL_ID.getProto());
     builder.setShouldDie(true);
     builder.setOutputTable("");
@@ -151,6 +153,7 @@ public class TaskSchedulerImpl extends AbstractService
       req.getCallback().run(stopTaskRunnerReq);
     }
 
+    LOG.info("Task Scheduler stopped");
     super.stop();
   }
 
@@ -301,7 +304,7 @@ public class TaskSchedulerImpl extends AbstractService
       while (it.hasNext() && leafTasks.size() > 0) {
         taskRequest = it.next();
         ContainerProxy container = context.getContainer(taskRequest.getContainerId());
-        String hostName = container.getHostName();
+        String hostName = container.getTaskHostName();
 
         QueryUnitAttemptId attemptId = null;
 
@@ -360,7 +363,7 @@ public class TaskSchedulerImpl extends AbstractService
 
           context.getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
               taskRequest.getContainerId(),
-              container.getHostName(), container.getPullServerPort()));
+              container.getTaskHostName(), container.getTaskPort()));
           AssignedRequest.add(attemptId);
 
           totalAssigned++;
@@ -411,7 +414,7 @@ public class TaskSchedulerImpl extends AbstractService
           ContainerProxy container = context.getContainer(
               taskRequest.getContainerId());
           context.getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
-              taskRequest.getContainerId(), container.getHostName(), container.getPullServerPort()));
+              taskRequest.getContainerId(), container.getTaskHostName(), container.getTaskPort()));
           taskRequest.getCallback().run(taskAssign.getProto());
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/cluster/WorkerListener.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/cluster/WorkerListener.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/cluster/WorkerListener.java
deleted file mode 100644
index b5bb84c..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/cluster/WorkerListener.java
+++ /dev/null
@@ -1,148 +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.cluster;
-
-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.net.NetUtils;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.TajoIdProtos.QueryUnitAttemptIdProto;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.MasterWorkerProtos.QueryUnitRequestProto;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskCompletionReport;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskFatalErrorReport;
-import org.apache.tajo.engine.MasterWorkerProtos.TaskStatusProto;
-import org.apache.tajo.ipc.MasterWorkerProtocol;
-import org.apache.tajo.ipc.MasterWorkerProtocol.MasterWorkerProtocolService;
-import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.event.TaskAttemptStatusUpdateEvent;
-import org.apache.tajo.master.event.TaskCompletionEvent;
-import org.apache.tajo.master.event.TaskFatalErrorEvent;
-import org.apache.tajo.rpc.ProtoAsyncRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
-
-import java.net.InetSocketAddress;
-
-public class WorkerListener extends AbstractService
-    implements MasterWorkerProtocolService.Interface {
-  
-  private final static Log LOG = LogFactory.getLog(WorkerListener.class);
-  private MasterContext context;
-  private ProtoAsyncRpcServer rpcServer;
-  private InetSocketAddress bindAddr;
-  private String addr;
-  
-  public WorkerListener(final MasterContext context) throws Exception {
-    super(WorkerListener.class.getName());
-    this.context = context;
-
-    String confMasterAddr = context.getConf().getVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS);
-    InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterAddr);
-    if (initIsa.getAddress() == null) {
-      throw new IllegalArgumentException("Failed resolve of " + initIsa);
-    }
-    try {
-      this.rpcServer = new ProtoAsyncRpcServer(MasterWorkerProtocol.class,
-          this, initIsa);
-    } catch (Exception e) {
-      LOG.error(e);
-    }
-    this.rpcServer.start();
-    this.bindAddr = rpcServer.getBindAddress();
-    this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
-
-    // Setup RPC server
-    // Get the master address
-    LOG.info(WorkerListener.class.getSimpleName() + " is bind to " + addr);
-    context.getConf().setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
-  }
-
-  @Override
-  public void init(Configuration conf) {
-    super.init(conf);
-  }
-
-  @Override
-  public void start() {
-    super.start();
-  }
-
-  @Override
-  public void stop() {
-    rpcServer.shutdown();
-    super.stop();
-  }
-  
-  public InetSocketAddress getBindAddress() {
-    return this.bindAddr;
-  }
-  
-  public String getAddress() {
-    return this.addr;
-  }
-
-  static BoolProto TRUE_PROTO = BoolProto.newBuilder().setValue(true).build();
-
-  @Override
-  public void getTask(RpcController controller, ContainerIdProto request,
-                      RpcCallback<QueryUnitRequestProto> done) {
-    //LOG.info("Get TaskRequest from " + request.getHost());
-    //context.getEventHandler().handle(new TaskRequestEvent(new NodeIdPBImpl(request), done));
-  }
-
-  @Override
-  public void statusUpdate(RpcController controller, TaskStatusProto request,
-                           RpcCallback<BoolProto> done) {
-    QueryUnitAttemptId attemptId = new QueryUnitAttemptId(request.getId());
-    context.getEventHandler().handle(new TaskAttemptStatusUpdateEvent(attemptId,
-        request));
-    done.run(TRUE_PROTO);
-  }
-
-  @Override
-  public void ping(RpcController controller,
-                   QueryUnitAttemptIdProto attemptIdProto,
-                   RpcCallback<BoolProto> done) {
-    QueryUnitAttemptId attemptId = new QueryUnitAttemptId(attemptIdProto);
-    context.getQuery(attemptId.getQueryId()).getContext().getSubQuery(attemptId.getSubQueryId()).
-        getQueryUnit(attemptId.getQueryUnitId()).getAttempt(attemptId).
-        resetExpireTime();
-    done.run(TRUE_PROTO);
-  }
-
-  @Override
-  public void fatalError(RpcController controller, TaskFatalErrorReport report,
-                         RpcCallback<BoolProto> done) {
-    context.getEventHandler().handle(new TaskFatalErrorEvent(report));
-    done.run(TRUE_PROTO);
-  }
-
-  @Override
-  public void done(RpcController controller, TaskCompletionReport report,
-                       RpcCallback<BoolProto> done) {
-    context.getEventHandler().handle(new TaskCompletionEvent(report));
-    done.run(TRUE_PROTO);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 4032c67..26c7231 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
@@ -19,7 +19,7 @@
 package org.apache.tajo.master.event;
 
 import org.apache.tajo.SubQueryId;
-import org.apache.tajo.master.SubQueryState;
+import org.apache.tajo.master.querymaster.SubQueryState;
 
 public class SubQueryCompletedEvent extends QueryEvent {
   private final SubQueryId subQueryId;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 3191639..d85d4f2 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
@@ -20,7 +20,7 @@ package org.apache.tajo.master.event;
 
 import org.apache.tajo.SubQueryId;
 import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.master.SubQueryState;
+import org.apache.tajo.master.querymaster.SubQueryState;
 
 public class SubQuerySucceeEvent extends SubQueryCompletedEvent {
   private final TableMeta tableMeta;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 6409b43..bc84011 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.engine.MasterWorkerProtos.TaskStatusProto;
+import org.apache.tajo.ipc.QueryMasterProtocol.TaskStatusProto;
 
 public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent {
   private final TaskStatusProto status;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 b36d69c..e3a4b5f 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.engine.MasterWorkerProtos.TaskCompletionReport;
+import org.apache.tajo.ipc.QueryMasterProtocol.TaskCompletionReport;
 
 public class TaskCompletionEvent extends TaskAttemptEvent {
   private TaskCompletionReport report;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 3d1c78d..06fb392 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.engine.MasterWorkerProtos.TaskFatalErrorReport;
+import org.apache.tajo.ipc.QueryMasterProtocol.TaskFatalErrorReport;
 
 public class TaskFatalErrorEvent extends TaskAttemptEvent {
   private TaskFatalErrorReport report;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 25a8a14..166e103 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,7 @@ 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.engine.MasterWorkerProtos.QueryUnitRequestProto;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryUnitRequestProto;
 import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
 
 public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..3179abf
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -0,0 +1,413 @@
+/**
+ * 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.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.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;
+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;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+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 final EventHandler eventHandler;
+  private final MasterPlan plan;
+  private final StorageManager sm;
+  private QueryContext context;
+  private ExecutionBlockCursor cursor;
+
+  // Query Status
+  private final QueryId id;
+  private long appSubmitTime;
+  private long startTime;
+  private long initializationTime;
+  private long finishTime;
+  private TableDesc resultDesc;
+  private int completedSubQueryCount = 0;
+  private final List<String> diagnostics = new ArrayList<String>();
+
+  // Internal Variables
+  private final Lock readLock;
+  private final Lock writeLock;
+  private int priority = 100;
+
+  // State Machine
+  private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
+
+  private static final StateMachineFactory
+      <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
+      new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
+          (QueryState.QUERY_NEW)
+
+      .addTransition(QueryState.QUERY_NEW,
+          EnumSet.of(QueryState.QUERY_INIT, QueryState.QUERY_FAILED),
+          QueryEventType.INIT, new InitTransition())
+
+      .addTransition(QueryState.QUERY_INIT, QueryState.QUERY_RUNNING,
+          QueryEventType.START, new StartTransition())
+
+      .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+          QueryEventType.INIT_COMPLETED, new InitCompleteTransition())
+      .addTransition(QueryState.QUERY_RUNNING,
+          EnumSet.of(QueryState.QUERY_RUNNING, QueryState.QUERY_SUCCEEDED,
+              QueryState.QUERY_FAILED),
+          QueryEventType.SUBQUERY_COMPLETED,
+          new SubQueryCompletedTransition())
+      .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
+          QueryEventType.INTERNAL_ERROR, new InternalErrorTransition())
+       .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+          QueryEventType.INTERNAL_ERROR)
+
+      .installTopology();
+
+  public Query(final QueryContext context, final QueryId id, Clock clock,
+               final long appSubmitTime,
+               final String queryStr,
+               final EventHandler eventHandler,
+               final MasterPlan plan,
+               final StorageManager sm) {
+    this.context = context;
+    this.conf = context.getConf();
+    this.id = id;
+    this.clock = clock;
+    this.appSubmitTime = appSubmitTime;
+    this.queryStr = queryStr;
+    subqueries = Maps.newHashMap();
+    this.eventHandler = eventHandler;
+    this.plan = plan;
+    this.sm = sm;
+    cursor = new ExecutionBlockCursor(plan);
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+
+    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()];
+      boolean finished = true;
+      for (SubQuery subquery: subqueries.values()) {
+        if (subquery.getState() != SubQueryState.NEW) {
+          subProgresses[idx] = subquery.getProgress();
+          if (finished && subquery.getState() != SubQueryState.SUCCEEDED) {
+            finished = false;
+          }
+        } else {
+          subProgresses[idx] = 0.0f;
+        }
+        idx++;
+      }
+
+      if (finished) {
+        return 1.0f;
+      }
+
+      float totalProgress = 0;
+      float proportion = 1.0f / (float)subqueries.size();
+
+      for (int i = 0; i < subProgresses.length; i++) {
+        totalProgress += subProgresses[i] * proportion;
+      }
+
+      return totalProgress;
+    }
+  }
+
+  public long getAppSubmitTime() {
+    return this.appSubmitTime;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public void setStartTime() {
+    startTime = clock.getTime();
+  }
+
+  public long getInitializationTime() {
+    return initializationTime;
+  }
+
+  public void setInitializationTime() {
+    initializationTime = clock.getTime();
+  }
+
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  public void setFinishTime() {
+    finishTime = clock.getTime();
+  }
+
+  public List<String> getDiagnostics() {
+    readLock.lock();
+    try {
+      return diagnostics;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  protected void addDiagnostic(String diag) {
+    diagnostics.add(diag);
+  }
+
+  public TableDesc getResultDesc() {
+    return resultDesc;
+  }
+
+  public void setResultDesc(TableDesc desc) {
+    resultDesc = desc;
+  }
+
+  public MasterPlan getPlan() {
+    return plan;
+  }
+
+  public StateMachine<QueryState, QueryEventType, QueryEvent> getStateMachine() {
+    return stateMachine;
+  }
+  
+  public void addSubQuery(SubQuery subquery) {
+    subqueries.put(subquery.getId(), subquery);
+  }
+  
+  public QueryId getId() {
+    return this.id;
+  }
+  
+  public SubQuery getSubQuery(SubQueryId id) {
+    return this.subqueries.get(id);
+  }
+
+  public QueryState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public ExecutionBlockCursor getExecutionBlockCursor() {
+    return cursor;
+  }
+
+  static class InitTransition
+      implements MultipleArcTransition<Query, QueryEvent, QueryState> {
+
+    @Override
+    public QueryState transition(Query query, QueryEvent queryEvent) {
+      query.setStartTime();
+      query.context.setState(QueryState.QUERY_INIT);
+      return QueryState.QUERY_INIT;
+    }
+  }
+
+  public static class StartTransition
+      implements SingleArcTransition<Query, QueryEvent> {
+
+    @Override
+    public void transition(Query query, QueryEvent queryEvent) {
+      SubQuery subQuery = new SubQuery(query.context, query.getExecutionBlockCursor().nextBlock(),
+          query.sm);
+      subQuery.setPriority(query.priority--);
+      query.addSubQuery(subQuery);
+      LOG.info("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+      subQuery.handle(new SubQueryEvent(subQuery.getId(),
+          SubQueryEventType.SQ_INIT));
+    }
+  }
+
+  public static class SubQueryCompletedTransition implements
+      MultipleArcTransition<Query, QueryEvent, QueryState> {
+
+    @Override
+    public QueryState transition(Query query, QueryEvent event) {
+      // increase the count for completed subqueries
+      query.completedSubQueryCount++;
+      SubQueryCompletedEvent castEvent = (SubQueryCompletedEvent) event;
+      ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+
+      // if the subquery is succeeded
+      if (castEvent.getFinalState() == SubQueryState.SUCCEEDED) {
+        if (cursor.hasNext()) {
+          SubQuery nextSubQuery = new SubQuery(query.context, cursor.nextBlock(), query.sm);
+          nextSubQuery.setPriority(query.priority--);
+          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());
+          return query.checkQueryForCompleted();
+
+        } else { // Finish a query
+          if (query.checkQueryForCompleted() == QueryState.QUERY_SUCCEEDED) {
+            SubQuery subQuery = query.getSubQuery(castEvent.getSubQueryId());
+            TableDesc desc = new TableDescImpl(query.conf.getOutputTable(),
+                subQuery.getTableMeta(), query.context.getOutputPath());
+            query.setResultDesc(desc);
+            try {
+              query.writeStat(query.context.getOutputPath(), subQuery);
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+            query.eventHandler.handle(new QueryFinishEvent(query.getId()));
+
+            if (query.context.isCreateTableQuery()) {
+              // TOOD move to QueryMasterManager
+              //query.context.getCatalog().addTable(desc);
+            }
+          }
+
+          return query.finished(QueryState.QUERY_SUCCEEDED);
+        }
+      } else {
+        // if at least one subquery is failed, the query is also failed.
+        return QueryState.QUERY_FAILED;
+      }
+    }
+  }
+
+  private static class DiagnosticsUpdateTransition implements
+      SingleArcTransition<Query, QueryEvent> {
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event)
+          .getDiagnosticUpdate());
+    }
+  }
+
+  private static class InitCompleteTransition implements
+      SingleArcTransition<Query, QueryEvent> {
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      if (query.initializationTime == 0) {
+        query.setInitializationTime();
+      }
+    }
+  }
+
+  private static class InternalErrorTransition
+      implements SingleArcTransition<Query, QueryEvent> {
+
+    @Override
+    public void transition(Query query, QueryEvent event) {
+      query.finished(QueryState.QUERY_ERROR);
+    }
+  }
+
+  public QueryState finished(QueryState finalState) {
+    setFinishTime();
+    context.setState(finalState);
+    return finalState;
+  }
+
+  /**
+   * Check if all subqueries of the query are completed
+   * @return QueryState.QUERY_SUCCEEDED if all subqueries are completed.
+   */
+  QueryState checkQueryForCompleted() {
+    if (completedSubQueryCount == subqueries.size()) {
+      return QueryState.QUERY_SUCCEEDED;
+    }
+    return getState();
+  }
+
+
+  @Override
+  public void handle(QueryEvent event) {
+    LOG.info("Processing " + event.getQueryId() + " of type " + event.getType());
+    try {
+      writeLock.lock();
+      QueryState oldState = getState();
+      try {
+        getStateMachine().doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state", e);
+        eventHandler.handle(new QueryEvent(this.id,
+            QueryEventType.INTERNAL_ERROR));
+      }
+
+      //notify the eventhandler of state change
+      if (oldState != getState()) {
+        LOG.info(id + " Query Transitioned from " + oldState + " to "
+            + getState());
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  private void writeStat(Path outputPath, SubQuery subQuery)
+      throws IOException {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    sm.writeTableMeta(outputPath, subQuery.getTableMeta());
+  }
+}


[3/8] TAJO-91: Launch QueryMaster on NodeManager per query. (hyoungjunkim via hyunsik)

Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..5443858
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
@@ -0,0 +1,344 @@
+/**
+ * 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.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+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.master.event.*;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+
+import java.util.*;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
+
+  private static final Log LOG = LogFactory.getLog(QueryUnitAttempt.class);
+
+  private final static int EXPIRE_TIME = 15000;
+
+  private final QueryUnitAttemptId id;
+  private final QueryUnit queryUnit;
+  final EventHandler eventHandler;
+
+  private String hostName;
+  private int port;
+  private int expire;
+
+  private final Lock readLock;
+  private final Lock writeLock;
+
+  private final List<String> diagnostics = new ArrayList<String>();
+
+  private static final StateMachineFactory
+      <QueryUnitAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+      stateMachineFactory = new StateMachineFactory
+      <QueryUnitAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+      (TaskAttemptState.TA_NEW)
+
+      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
+          TaskAttemptEventType.TA_SCHEDULE, new TaskAttemptScheduleTransition())
+      .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
+          TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition())
+
+      .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_ASSIGNED,
+          TaskAttemptEventType.TA_ASSIGNED, new LaunchTransition())
+
+      // from assigned
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED,
+          TaskAttemptEventType.TA_ASSIGNED, new AlreadyAssignedTransition())
+      .addTransition(TaskAttemptState.TA_ASSIGNED,
+          EnumSet.of(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILLED),
+          TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
+
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_DONE, new SucceededTransition())
+
+      .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+      // from running
+      .addTransition(TaskAttemptState.TA_RUNNING,
+          EnumSet.of(TaskAttemptState.TA_RUNNING),
+          TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
+
+      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_DONE, new SucceededTransition())
+
+      .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_UPDATE)
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+          TaskAttemptEventType.TA_DONE, new AlreadyDoneTransition())
+      .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_FAILED,
+          TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+      .installTopology();
+
+  private final StateMachine<TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+    stateMachine;
+
+
+  public QueryUnitAttempt(final QueryUnitAttemptId id, final QueryUnit queryUnit,
+                          final EventHandler eventHandler) {
+    this.id = id;
+    this.expire = QueryUnitAttempt.EXPIRE_TIME;
+    this.queryUnit = queryUnit;
+    this.eventHandler = eventHandler;
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+
+    stateMachine = stateMachineFactory.make(this);
+  }
+
+  public TaskAttemptState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public QueryUnitAttemptId getId() {
+    return this.id;
+  }
+
+  public boolean isLeafTask() {
+    return this.queryUnit.isLeafTask();
+  }
+
+  public QueryUnit getQueryUnit() {
+    return this.queryUnit;
+  }
+
+  public String getHost() {
+    return this.hostName;
+  }
+
+  public void setHost(String host) {
+    this.hostName = host;
+  }
+
+  public void setPullServerPort(int port) {
+    this.port = port;
+  }
+
+  public int getPullServerPort() {
+    return port;
+  }
+
+  public synchronized void setExpireTime(int expire) {
+    this.expire = expire;
+  }
+
+  public synchronized void updateExpireTime(int period) {
+    this.setExpireTime(this.expire - period);
+  }
+
+  public synchronized void resetExpireTime() {
+    this.setExpireTime(QueryUnitAttempt.EXPIRE_TIME);
+  }
+
+  public int getLeftTime() {
+    return this.expire;
+  }
+
+  private void fillTaskStatistics(TaskCompletionReport report) {
+    if (report.getPartitionsCount() > 0) {
+      this.getQueryUnit().setPartitions(report.getPartitionsList());
+
+      List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+      for (Partition p : report.getPartitionsList()) {
+        IntermediateEntry entry = new IntermediateEntry(getId().getQueryUnitId().getId(),
+            getId().getId(), p.getPartitionKey(), getHost(), getPullServerPort());
+        partitions.add(entry);
+      }
+      this.getQueryUnit().setIntermediateData(partitions);
+    }
+    if (report.hasResultStats()) {
+      this.getQueryUnit().setStats(new TableStat(report.getResultStats()));
+    }
+  }
+
+  private static class TaskAttemptScheduleTransition implements
+    SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent taskAttemptEvent) {
+
+      if (taskAttempt.isLeafTask()
+          && taskAttempt.getQueryUnit().getScanNodes().length == 1) {
+        Set<String> racks = new HashSet<String>();
+        for (String host : taskAttempt.getQueryUnit().getDataLocations()) {
+          racks.add(RackResolver.resolve(host).getNetworkLocation());
+        }
+
+        taskAttempt.eventHandler.handle(new TaskScheduleEvent(
+            taskAttempt.getId(), EventType.T_SCHEDULE, true,
+            taskAttempt.getQueryUnit().getDataLocations(),
+            racks.toArray(new String[racks.size()])
+        ));
+      } else {
+        taskAttempt.eventHandler.handle(new TaskScheduleEvent(
+            taskAttempt.getId(), EventType.T_SCHEDULE,
+            false,
+            null,
+            null
+        ));
+      }
+    }
+  }
+
+  private static class LaunchTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent> {
+
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent event) {
+      TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event;
+      taskAttempt.setHost(castEvent.getHostName());
+      taskAttempt.setPullServerPort(castEvent.getPullServerPort());
+      taskAttempt.eventHandler.handle(
+          new TaskTAttemptEvent(taskAttempt.getId(),
+              TaskEventType.T_ATTEMPT_LAUNCHED));
+    }
+  }
+
+  private static class StatusUpdateTransition
+      implements MultipleArcTransition<QueryUnitAttempt, TaskAttemptEvent, TaskAttemptState> {
+
+    @Override
+    public TaskAttemptState transition(QueryUnitAttempt taskAttempt,
+                                       TaskAttemptEvent event) {
+      TaskAttemptStatusUpdateEvent updateEvent =
+          (TaskAttemptStatusUpdateEvent) event;
+
+      switch (updateEvent.getStatus().getState()) {
+        case TA_PENDING:
+        case TA_RUNNING:
+          return TaskAttemptState.TA_RUNNING;
+
+        default:
+          return taskAttempt.getState();
+      }
+    }
+  }
+
+  private void addDiagnosticInfo(String diag) {
+    if (diag != null && !diag.equals("")) {
+      diagnostics.add(diag);
+    }
+  }
+
+  private static class AlreadyAssignedTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+
+    @Override
+    public void transition(QueryUnitAttempt queryUnitAttempt,
+                           TaskAttemptEvent taskAttemptEvent) {
+      LOG.info(">>>>>>>>> Already Assigned: " + queryUnitAttempt.getId());
+    }
+  }
+
+  private static class AlreadyDoneTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+
+    @Override
+    public void transition(QueryUnitAttempt queryUnitAttempt,
+                           TaskAttemptEvent taskAttemptEvent) {
+      LOG.info(">>>>>>>>> Already Done: " + queryUnitAttempt.getId());
+    }
+  }
+
+  private static class SucceededTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent event) {
+      TaskCompletionReport report = ((TaskCompletionEvent)event).getReport();
+
+      taskAttempt.fillTaskStatistics(report);
+      taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(),
+          TaskEventType.T_ATTEMPT_SUCCEEDED));
+    }
+  }
+
+  private static class FailedTransition
+      implements SingleArcTransition<QueryUnitAttempt, TaskAttemptEvent>{
+    @Override
+    public void transition(QueryUnitAttempt taskAttempt,
+                           TaskAttemptEvent event) {
+      TaskFatalErrorEvent errorEvent = (TaskFatalErrorEvent) event;
+      taskAttempt.eventHandler.handle(
+          new TaskTAttemptEvent(taskAttempt.getId(),
+              TaskEventType.T_ATTEMPT_FAILED));
+      LOG.error("FROM " + taskAttempt.getHost() + " >> "
+          + errorEvent.errorMessage());
+      taskAttempt.addDiagnosticInfo(errorEvent.errorMessage());
+    }
+  }
+
+  @Override
+  public void handle(TaskAttemptEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getTaskAttemptId() + " of type "
+          + event.getType());
+    }
+    try {
+      writeLock.lock();
+      TaskAttemptState oldState = getState();
+      try {
+        stateMachine.doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state of "
+            + event.getTaskAttemptId() + ")", e);
+        eventHandler.handle(new QueryEvent(getId().getQueryId(),
+            QueryEventType.INTERNAL_ERROR));
+      }
+
+      //notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+       if (oldState != getState()) {
+          LOG.debug(id + " TaskAttempt Transitioned from " + oldState + " to "
+              + getState());
+        }
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..3957d57
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -0,0 +1,584 @@
+/**
+ * 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.fs.Path;
+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;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
+import org.apache.tajo.engine.planner.UniformRangePartition;
+import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.master.ExecutionBlock;
+import org.apache.tajo.master.ExecutionBlock.PartitionType;
+import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.storage.Fragment;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.math.BigDecimal;
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+
+/**
+ * Repartitioner creates non-leaf tasks and shuffles intermediate data.
+ * It supports two repartition methods, such as hash and range repartition.
+ */
+public class Repartitioner {
+  private static final Log LOG = LogFactory.getLog(Repartitioner.class);
+
+  private static int HTTP_REQUEST_MAXIMUM_LENGTH = 1900;
+
+  public static QueryUnit[] createJoinTasks(SubQuery subQuery)
+      throws IOException {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    //CatalogService catalog = subQuery.getContext().getCatalog();
+
+    ScanNode[] scans = execBlock.getScanNodes();
+    Path tablePath;
+    Fragment [] fragments = new Fragment[2];
+    TableStat [] stats = new TableStat[2];
+
+    // initialize variables from the child operators
+    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)) {
+        tablePath = subQuery.getStorageManager().getTablePath(scans[i].getTableId());
+      } else {
+        tablePath = tableDesc.getPath();
+      }
+
+      if (scans[i].isLocal()) { // it only requires a dummy fragment.
+        fragments[i] = new Fragment(scans[i].getTableId(), tablePath,
+            CatalogUtil.newTableMeta(scans[i].getInSchema(), StoreType.CSV),
+            0, 0, null);
+      } else {
+        fragments[i] = subQuery.getStorageManager().getSplits(scans[i].getTableId(),
+                tableDesc.getMeta(),
+            new Path(tablePath, "data")).get(0);
+      }
+
+      // Getting a table stat for each scan
+      stats[i] = subQuery.getChildQuery(scans[i]).getTableStat();
+    }
+
+    // Assigning either fragments or fetch urls to query units
+    QueryUnit [] tasks;
+    if (scans[0].isBroadcast() || scans[1].isBroadcast()) {
+      tasks = new QueryUnit[1];
+      tasks[0] = new QueryUnit(QueryIdFactory.newQueryUnitId(subQuery.getId(), 0),
+          false, subQuery.getEventHandler());
+      tasks[0].setLogicalPlan(execBlock.getPlan());
+      tasks[0].setFragment(scans[0].getTableId(), fragments[0]);
+      tasks[0].setFragment(scans[1].getTableId(), fragments[1]);
+    } else {
+      // The hash map is modeling as follows:
+      // <Partition Id, <Table Name, Intermediate Data>>
+      Map<Integer, Map<String, List<IntermediateEntry>>> hashEntries =
+          new HashMap<Integer, Map<String, List<IntermediateEntry>>>();
+
+      // Grouping IntermediateData by a partition key and a table name
+      for (ScanNode scan : scans) {
+        SubQuery childSubQuery = subQuery.getChildQuery(scan);
+        for (QueryUnit task : childSubQuery.getQueryUnits()) {
+          if (task.getIntermediateData() != null) {
+            for (IntermediateEntry intermEntry : task.getIntermediateData()) {
+              if (hashEntries.containsKey(intermEntry.getPartitionId())) {
+                Map<String, List<IntermediateEntry>> tbNameToInterm =
+                    hashEntries.get(intermEntry.getPartitionId());
+
+                if (tbNameToInterm.containsKey(scan.getTableId())) {
+                  tbNameToInterm.get(scan.getTableId()).add(intermEntry);
+                } else {
+                  tbNameToInterm.put(scan.getTableId(), TUtil.newList(intermEntry));
+                }
+              } else {
+                Map<String, List<IntermediateEntry>> tbNameToInterm =
+                    new HashMap<String, List<IntermediateEntry>>();
+                tbNameToInterm.put(scan.getTableId(), TUtil.newList(intermEntry));
+                hashEntries.put(intermEntry.getPartitionId(), tbNameToInterm);
+              }
+            }
+          }
+        }
+      }
+
+      LOG.info("Outer Intermediate Volume: " + stats[0].getNumBytes());
+      LOG.info("Inner Intermediate Volume: " + stats[1].getNumBytes());
+
+      // Getting the desire number of join tasks according to the volumn
+      // of a larger table
+      int largerIdx = stats[0].getNumBytes() >= stats[1].getNumBytes() ? 0 : 1;
+      int desireJoinTaskVolumn = subQuery.getContext().getConf().
+          getIntVar(ConfVars.JOIN_TASK_VOLUME);
+
+      // calculate the number of tasks according to the data size
+      int mb = (int) Math.ceil((double)stats[largerIdx].getNumBytes() / 1048576);
+      LOG.info("Larger intermediate data is approximately " + mb + " MB");
+      // determine the number of task per 64MB
+      int maxTaskNum = (int) Math.ceil((double)mb / desireJoinTaskVolumn);
+      LOG.info("The calculated number of tasks is " + maxTaskNum);
+      LOG.info("The number of total partition keys is " + hashEntries.size());
+      // the number of join tasks cannot be larger than the number of
+      // distinct partition ids.
+      int joinTaskNum = Math.min(maxTaskNum, hashEntries.size());
+      LOG.info("The determined number of join tasks is " + joinTaskNum);
+      QueryUnit [] createdTasks = newEmptyJoinTask(subQuery, fragments, joinTaskNum);
+
+      // Assign partitions to tasks in a round robin manner.
+      int i = 0;
+      for (Entry<Integer, Map<String, List<IntermediateEntry>>> entry
+          : hashEntries.entrySet()) {
+        addJoinPartition(createdTasks[i++], subQuery, entry.getKey(), entry.getValue());
+        if (i >= joinTaskNum) {
+          i = 0;
+        }
+      }
+
+      List<QueryUnit> filteredTasks = new ArrayList<QueryUnit>();
+      for (QueryUnit task : createdTasks) {
+        // if there are at least two fetches, the join is possible.
+        if (task.getFetches().size() > 1) {
+          filteredTasks.add(task);
+        }
+      }
+
+      tasks = filteredTasks.toArray(new QueryUnit[filteredTasks.size()]);
+    }
+
+    return tasks;
+  }
+
+  private static QueryUnit [] newEmptyJoinTask(SubQuery subQuery, Fragment [] fragments, int taskNum) {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    QueryUnit [] tasks = new QueryUnit[taskNum];
+    for (int i = 0; i < taskNum; i++) {
+      tasks[i] = new QueryUnit(
+          QueryIdFactory.newQueryUnitId(subQuery.getId(), i), execBlock.isLeafBlock(),
+          subQuery.getEventHandler());
+      tasks[i].setLogicalPlan(execBlock.getPlan());
+      for (Fragment fragment : fragments) {
+        tasks[i].setFragment2(fragment);
+      }
+    }
+
+    return tasks;
+  }
+
+  private static void addJoinPartition(QueryUnit task, SubQuery subQuery, int partitionId,
+                                       Map<String, List<IntermediateEntry>> grouppedPartitions) {
+
+    for (ScanNode scanNode : subQuery.getBlock().getScanNodes()) {
+      Map<String, List<IntermediateEntry>> requests;
+      if (grouppedPartitions.containsKey(scanNode.getTableId())) {
+          requests = mergeHashPartitionRequest(grouppedPartitions.get(scanNode.getTableId()));
+      } else {
+        return;
+      }
+      Set<URI> fetchURIs = TUtil.newHashSet();
+      for (Entry<String, List<IntermediateEntry>> requestPerNode : requests.entrySet()) {
+        Collection<URI> uris = createHashFetchURL(requestPerNode.getKey(),
+            subQuery.getChildQuery(scanNode).getId(),
+            partitionId, PartitionType.HASH,
+            requestPerNode.getValue());
+        fetchURIs.addAll(uris);
+      }
+      task.addFetches(scanNode.getTableId(), fetchURIs);
+    }
+  }
+
+  /**
+   * This method merges the partition request associated with the pullserver's address.
+   * It reduces the number of TCP connections.
+   *
+   * @return key: pullserver's address, value: a list of requests
+   */
+  private static Map<String, List<IntermediateEntry>> mergeHashPartitionRequest(
+      List<IntermediateEntry> partitions) {
+    Map<String, List<IntermediateEntry>> mergedPartitions =
+        new HashMap<String, List<IntermediateEntry>>();
+    for (IntermediateEntry partition : partitions) {
+      if (mergedPartitions.containsKey(partition.getPullAddress())) {
+        mergedPartitions.get(partition.getPullAddress()).add(partition);
+      } else {
+        mergedPartitions.put(partition.getPullAddress(), TUtil.newList(partition));
+      }
+    }
+
+    return mergedPartitions;
+  }
+
+  public static QueryUnit [] createNonLeafTask(SubQuery subQuery,
+                                               SubQuery childSubQuery,
+                                               int maxNum)
+      throws InternalException {
+    ExecutionBlock childExecBlock = childSubQuery.getBlock();
+    if (childExecBlock.getPartitionType() == PartitionType.HASH) {
+      return createHashPartitionedTasks(subQuery, childSubQuery, maxNum);
+    } else if (childExecBlock.getPartitionType() == PartitionType.RANGE) {
+      return createRangePartitionedTasks(subQuery, childSubQuery, maxNum);
+    } else {
+      throw new InternalException("Cannot support partition type");
+    }
+  }
+
+  public static QueryUnit [] createRangePartitionedTasks(SubQuery subQuery,
+                                                         SubQuery childSubQuery,
+                                                         int maxNum)
+      throws InternalException {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    TableStat stat = childSubQuery.getTableStat();
+    if (stat.getNumRows() == 0) {
+      return new QueryUnit[0];
+    }
+
+    ScanNode scan = execBlock.getScanNodes()[0];
+    Path tablePath;
+    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableId());
+
+    StoreTableNode store = (StoreTableNode) childSubQuery.getBlock().getPlan();
+    SortNode sort = (SortNode) store.getSubNode();
+    SortSpec[] sortSpecs = sort.getSortKeys();
+    Schema sortSchema = PlannerUtil.sortSpecsToSchema(sort.getSortKeys());
+
+    // calculate the number of maximum query ranges
+    TupleRange mergedRange =
+        TupleUtil.columnStatToRange(sort.getOutSchema(),
+            sortSchema, stat.getColumnStats());
+    RangePartitionAlgorithm partitioner =
+        new UniformRangePartition(sortSchema, mergedRange);
+    BigDecimal card = partitioner.getTotalCardinality();
+
+    // if the number of the range cardinality is less than the desired number of tasks,
+    // we set the the number of tasks to the number of range cardinality.
+    int determinedTaskNum;
+    if (card.compareTo(new BigDecimal(maxNum)) < 0) {
+      LOG.info("The range cardinality (" + card
+          + ") is less then the desired number of tasks (" + maxNum + ")");
+      determinedTaskNum = card.intValue();
+    } else {
+      determinedTaskNum = maxNum;
+    }
+
+    LOG.info("Try to divide " + mergedRange + " into " + determinedTaskNum +
+        " sub ranges (total units: " + determinedTaskNum + ")");
+    TupleRange [] ranges = partitioner.partition(determinedTaskNum);
+
+    Fragment dummyFragment = new Fragment(scan.getTableId(), tablePath,
+        CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
+        0, 0, null);
+
+    List<String> basicFetchURIs = new ArrayList<String>();
+
+    SubQuery child = childSubQuery.getContext().getSubQuery(
+        subQuery.getBlock().getChildBlock(scan).getId());
+    for (QueryUnit qu : child.getQueryUnits()) {
+      for (IntermediateEntry p : qu.getIntermediateData()) {
+        String uri = createBasicFetchUri(p.getPullHost(), p.getPullPort(),
+            childSubQuery.getId(), p.taskId, p.attemptId);
+        basicFetchURIs.add(uri);
+      }
+    }
+
+    boolean ascendingFirstKey = sortSpecs[0].isAscending();
+    SortedMap<TupleRange, Set<URI>> map;
+    if (ascendingFirstKey) {
+      map = new TreeMap<TupleRange, Set<URI>>();
+    } else {
+      map = new TreeMap<TupleRange, Set<URI>>(new TupleRange.DescendingTupleRangeComparator());
+    }
+
+    Set<URI> uris;
+    try {
+      for (int i = 0; i < ranges.length; i++) {
+        uris = new HashSet<URI>();
+        for (String uri: basicFetchURIs) {
+          String rangeParam = TupleUtil.rangeToQuery(sortSchema, ranges[i],
+              ascendingFirstKey, ascendingFirstKey ? i == (ranges.length - 1) : i == 0);
+          URI finalUri = URI.create(uri + "&" + rangeParam);
+          uris.add(finalUri);
+        }
+        map.put(ranges[i], uris);
+      }
+
+    } catch (UnsupportedEncodingException e) {
+      LOG.error(e);
+    }
+
+    QueryUnit [] tasks = createEmptyNonLeafTasks(subQuery, determinedTaskNum, dummyFragment);
+    assignPartitionByRoundRobin(map, scan.getTableId(), tasks);
+    return tasks;
+  }
+
+  public static QueryUnit [] assignPartitionByRoundRobin(Map<?, Set<URI>> partitions,
+                                               String tableName, QueryUnit [] tasks) {
+    int tid = 0;
+    for (Entry<?, Set<URI>> entry : partitions.entrySet()) {
+      for (URI uri : entry.getValue()) {
+        tasks[tid].addFetch(tableName, uri);
+      }
+
+      if (tid >= tasks.length) {
+        tid = 0;
+      } else {
+        tid ++;
+      }
+    }
+
+    return tasks;
+  }
+
+  public static String createBasicFetchUri(String hostName, int port,
+                                           SubQueryId 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())
+        .append("&").append("ta=").append(taskId).append("_").append(attemptId)
+        .append("&").append("p=0")
+        .append("&").append("type=r");
+
+    return sb.toString();
+  }
+
+  public static QueryUnit [] createHashPartitionedTasks(SubQuery subQuery,
+                                                 SubQuery childSubQuery,
+                                                 int maxNum) {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    TableStat stat = childSubQuery.getTableStat();
+    if (stat.getNumRows() == 0) {
+      return new QueryUnit[0];
+    }
+
+    ScanNode scan = execBlock.getScanNodes()[0];
+    Path tablePath;
+    tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableId());
+
+    List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+    for (QueryUnit tasks : childSubQuery.getQueryUnits()) {
+      if (tasks.getIntermediateData() != null) {
+        partitions.addAll(tasks.getIntermediateData());
+      }
+    }
+
+    Fragment frag = new Fragment(scan.getTableId(), tablePath,
+        CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),
+        0, 0, null);
+
+    Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
+    Map<String, List<IntermediateEntry>> hashedByHost;
+    Map<Integer, List<URI>> finalFetchURI = new HashMap<Integer, List<URI>>();
+
+    for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
+      hashedByHost = hashByHost(interm.getValue());
+      for (Entry<String, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
+        Collection<URI> uris = createHashFetchURL(e.getKey(), childSubQuery.getId(),
+            interm.getKey(),
+            childSubQuery.getBlock().getPartitionType(), e.getValue());
+
+        if (finalFetchURI.containsKey(interm.getKey())) {
+          finalFetchURI.get(interm.getKey()).addAll(uris);
+        } else {
+          finalFetchURI.put(interm.getKey(), TUtil.newList(uris));
+        }
+      }
+    }
+
+    GroupbyNode groupby = (GroupbyNode) childSubQuery.getBlock().getStoreTableNode().
+        getSubNode();
+    // the number of tasks cannot exceed the number of merged fetch uris.
+    int determinedTaskNum = Math.min(maxNum, finalFetchURI.size());
+    if (groupby.getGroupingColumns().length == 0) {
+      determinedTaskNum = 1;
+    }
+
+    QueryUnit [] tasks = createEmptyNonLeafTasks(subQuery, determinedTaskNum, frag);
+
+    int tid = 0;
+    for (Entry<Integer, List<URI>> entry : finalFetchURI.entrySet()) {
+      for (URI uri : entry.getValue()) {
+        tasks[tid].addFetch(scan.getTableId(), uri);
+      }
+
+      tid ++;
+
+      if (tid == tasks.length) {
+       tid = 0;
+      }
+    }
+
+    return tasks;
+  }
+
+  public static Collection<URI> createHashFetchURL(String hostAndPort, SubQueryId childSid,
+                                       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=");
+    if (type == PartitionType.HASH) {
+      urlPrefix.append("h");
+    } else if (type == PartitionType.RANGE) {
+      urlPrefix.append("r");
+    }
+    urlPrefix.append("&ta=");
+
+    // If the get request is longer than 2000 characters,
+    // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long.
+    // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15
+    // The below code transforms a long request to multiple requests.
+    List<String> taskIdsParams = new ArrayList<String>();
+    boolean first = true;
+    StringBuilder taskIdListBuilder = new StringBuilder();
+    for (IntermediateEntry entry: entries) {
+      StringBuilder taskAttemptId = new StringBuilder();
+
+      if (!first) { // when comma is added?
+        taskAttemptId.append(",");
+      } else {
+        first = false;
+      }
+
+      taskAttemptId.append(entry.getTaskId()).append("_").
+          append(entry.getAttemptId());
+      if (taskIdListBuilder.length() + taskAttemptId.length()
+          > HTTP_REQUEST_MAXIMUM_LENGTH) {
+        taskIdsParams.add(taskIdListBuilder.toString());
+        taskIdListBuilder = new StringBuilder(entry.getTaskId() + "_" + entry.getAttemptId());
+      } else {
+        taskIdListBuilder.append(taskAttemptId);
+      }
+    }
+
+    // if the url params remain
+    if (taskIdListBuilder.length() > 0) {
+      taskIdsParams.add(taskIdListBuilder.toString());
+    }
+
+    Collection<URI> fetchURLs = new ArrayList<URI>();
+    for (String param : taskIdsParams) {
+      fetchURLs.add(URI.create(urlPrefix + param));
+    }
+
+    return fetchURLs;
+  }
+
+  public static Map<Integer, List<IntermediateEntry>> hashByKey(
+      List<IntermediateEntry> entries) {
+    Map<Integer, List<IntermediateEntry>> hashed = new HashMap<Integer, List<IntermediateEntry>>();
+    for (IntermediateEntry entry : entries) {
+      if (hashed.containsKey(entry.getPartitionId())) {
+        hashed.get(entry.getPartitionId()).add(entry);
+      } else {
+        hashed.put(entry.getPartitionId(), TUtil.newList(entry));
+      }
+    }
+
+    return hashed;
+  }
+
+  public static QueryUnit [] createEmptyNonLeafTasks(SubQuery subQuery, int num,
+                                                     Fragment frag) {
+    LogicalNode plan = subQuery.getBlock().getPlan();
+    QueryUnit [] tasks = new QueryUnit[num];
+    for (int i = 0; i < num; i++) {
+      tasks[i] = new QueryUnit(QueryIdFactory.newQueryUnitId(subQuery.getId(), i),
+          false, subQuery.getEventHandler());
+      tasks[i].setFragment2(frag);
+      tasks[i].setLogicalPlan(plan);
+    }
+    return tasks;
+  }
+
+  public static Map<String, List<IntermediateEntry>> hashByHost(
+      List<IntermediateEntry> entries) {
+    Map<String, List<IntermediateEntry>> hashed = new HashMap<String, List<IntermediateEntry>>();
+
+    String hostName;
+    for (IntermediateEntry entry : entries) {
+      hostName = entry.getPullHost() + ":" + entry.getPullPort();
+      if (hashed.containsKey(hostName)) {
+        hashed.get(hostName).add(entry);
+      } else {
+        hashed.put(hostName, TUtil.newList(entry));
+      }
+    }
+
+    return hashed;
+  }
+
+  public static SubQuery setPartitionNumberForTwoPhase(SubQuery subQuery, final int n) {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    Column[] keys = null;
+    // if the next query is join,
+    // set the partition number for the current logicalUnit
+    // TODO: the union handling is required when a join has unions as its child
+    ExecutionBlock parentBlock = execBlock.getParentBlock();
+    if (parentBlock != null) {
+      if (parentBlock.getStoreTableNode().getSubNode().getType() == ExprType.JOIN) {
+        execBlock.getStoreTableNode().setPartitions(execBlock.getPartitionType(),
+            execBlock.getStoreTableNode().getPartitionKeys(), n);
+        keys = execBlock.getStoreTableNode().getPartitionKeys();
+      }
+    }
+
+    StoreTableNode store = execBlock.getStoreTableNode();
+    // set the partition number for group by and sort
+    if (execBlock.getPartitionType() == PartitionType.HASH) {
+      if (store.getSubNode().getType() == ExprType.GROUP_BY) {
+        GroupbyNode groupby = (GroupbyNode)store.getSubNode();
+        keys = groupby.getGroupingColumns();
+      }
+    } else if (execBlock.getPartitionType() == PartitionType.RANGE) {
+      if (store.getSubNode().getType() == ExprType.SORT) {
+        SortNode sort = (SortNode)store.getSubNode();
+        keys = new Column[sort.getSortKeys().length];
+        for (int i = 0; i < keys.length; i++) {
+          keys[i] = sort.getSortKeys()[i].getSortKey();
+        }
+      }
+    }
+    if (keys != null) {
+      if (keys.length == 0) {
+        store.setPartitions(execBlock.getPartitionType(), new Column[]{}, 1);
+      } else {
+        store.setPartitions(execBlock.getPartitionType(), keys, n);
+      }
+    } else {
+      store.setListPartition();
+    }
+    return subQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
new file mode 100644
index 0000000..305ef1b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -0,0 +1,766 @@
+/**
+ * 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.common.base.Preconditions;
+import com.google.common.collect.Lists;
+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.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.Priority;
+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.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;
+import org.apache.tajo.catalog.statistics.ColumnStat;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStat;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.logical.ExprType;
+import org.apache.tajo.engine.planner.logical.GroupbyNode;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.engine.planner.logical.StoreTableNode;
+import org.apache.tajo.master.ExecutionBlock;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+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;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+
+
+/**
+ * SubQuery plays a role in controlling an ExecutionBlock and is a finite state machine.
+ */
+public class SubQuery implements EventHandler<SubQueryEvent> {
+
+  private static final Log LOG = LogFactory.getLog(SubQuery.class);
+
+  private ExecutionBlock block;
+  private int priority;
+  private TableMeta meta;
+  private EventHandler eventHandler;
+  private final StorageManager sm;
+  private TaskSchedulerImpl taskScheduler;
+  private QueryContext context;
+
+  private long startTime;
+  private long finishTime;
+
+  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;
+
+  private StateMachineFactory<SubQuery, SubQueryState,
+      SubQueryEventType, SubQueryEvent> stateMachineFactory =
+      new StateMachineFactory <SubQuery, SubQueryState,
+          SubQueryEventType, SubQueryEvent> (SubQueryState.NEW)
+
+          .addTransition(SubQueryState.NEW,
+              EnumSet.of(SubQueryState.INIT, SubQueryState.FAILED, SubQueryState.SUCCEEDED),
+              SubQueryEventType.SQ_INIT, new InitAndRequestContainer())
+
+          .addTransition(SubQueryState.INIT, SubQueryState.CONTAINER_ALLOCATED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
+
+          .addTransition(SubQueryState.CONTAINER_ALLOCATED,
+              EnumSet.of(SubQueryState.RUNNING, SubQueryState.FAILED,
+                  SubQueryState.SUCCEEDED), SubQueryEventType.SQ_START, new StartTransition())
+          .addTransition(SubQueryState.CONTAINER_ALLOCATED, SubQueryState.CONTAINER_ALLOCATED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
+
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED, CONTAINER_LAUNCH_TRANSITION)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING, SubQueryEventType.SQ_START)
+          .addTransition(SubQueryState.RUNNING, SubQueryState.RUNNING,
+              SubQueryEventType.SQ_TASK_COMPLETED, new TaskCompletedTransition())
+          .addTransition(SubQueryState.RUNNING, SubQueryState.SUCCEEDED,
+              SubQueryEventType.SQ_SUBQUERY_COMPLETED, new SubQueryCompleteTransition())
+          .addTransition(SubQueryState.RUNNING, SubQueryState.FAILED,
+              SubQueryEventType.SQ_FAILED, new InternalErrorTransition())
+
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
+              SubQueryEventType.SQ_START)
+          .addTransition(SubQueryState.SUCCEEDED, SubQueryState.SUCCEEDED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED)
+
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_START)
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_CONTAINER_ALLOCATED)
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+                 SubQueryEventType.SQ_FAILED)
+          .addTransition(SubQueryState.FAILED, SubQueryState.FAILED,
+              SubQueryEventType.SQ_INTERNAL_ERROR);
+
+
+  private final Lock readLock;
+  private final Lock writeLock;
+
+  private int completedTaskCount = 0;
+
+  public SubQuery(QueryContext context, ExecutionBlock block, StorageManager sm) {
+    this.context = context;
+    this.block = block;
+    this.sm = sm;
+    this.eventHandler = context.getEventHandler();
+
+    ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    this.readLock = readWriteLock.readLock();
+    this.writeLock = readWriteLock.writeLock();
+    stateMachine = stateMachineFactory.make(this);
+  }
+
+  public QueryContext getContext() {
+    return context;
+  }
+
+  public EventHandler getEventHandler() {
+    return eventHandler;
+  }
+
+  public TaskScheduler getTaskScheduler() {
+    return taskScheduler;
+  }
+
+  public void setStartTime() {
+    startTime = context.getClock().getTime();
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public void setFinishTime() {
+    finishTime = context.getClock().getTime();
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public long getFinishTime() {
+    return this.finishTime;
+  }
+
+  public float getProgress() {
+    readLock.lock();
+    try {
+      if (getState() == SubQueryState.NEW) {
+        return 0;
+      } else {
+        if (completedTaskCount == 0) {
+          return 0.0f;
+        } else {
+          return (float)completedTaskCount / (float)tasks.size();
+        }
+      }
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  public ExecutionBlock getBlock() {
+    return block;
+  }
+
+  public void addTask(QueryUnit task) {
+    tasks.put(task.getId(), task);
+  }
+
+  public void abortSubQuery(SubQueryState finalState) {
+    // TODO -
+    // - committer.abortSubQuery(...)
+    // - record SubQuery Finish Time
+    // - CleanUp Tasks
+    // - Record History
+
+    eventHandler.handle(new SubQueryCompletedEvent(getId(), finalState));
+  }
+
+  public StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent> getStateMachine() {
+    return this.stateMachine;
+  }
+
+  public void setPriority(int priority) {
+    this.priority = priority;
+  }
+
+
+  public int getPriority() {
+    return this.priority;
+  }
+
+  public StorageManager getStorageManager() {
+    return sm;
+  }
+  
+  public SubQuery getChildQuery(ScanNode scanForChild) {
+    return context.getSubQuery(block.getChildBlock(scanForChild).getId());
+  }
+  
+  public SubQueryId getId() {
+    return block.getId();
+  }
+  
+  public QueryUnit[] getQueryUnits() {
+    return tasks.values().toArray(new QueryUnit[tasks.size()]);
+  }
+  
+  public QueryUnit getQueryUnit(QueryUnitId qid) {
+    return tasks.get(qid);
+  }
+
+  public void setTableMeta(TableMeta meta) {
+    this.meta = meta;
+  }
+
+  @SuppressWarnings("UnusedDeclaration")
+  public TableMeta getTableMeta() {
+    return meta;
+  }
+
+  public TableStat getTableStat() {
+    return this.meta.getStat();
+  }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(this.getId());
+    return sb.toString();
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o instanceof SubQuery) {
+      SubQuery other = (SubQuery)o;
+      return getId().equals(other.getId());
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    return getId().hashCode();
+  }
+  
+  public int compareTo(SubQuery other) {
+    return getId().compareTo(other.getId());
+  }
+
+  public SubQueryState getState() {
+    readLock.lock();
+    try {
+      return stateMachine.getCurrentState();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  private static TableStat computeStatFromUnionBlock(SubQuery unit) {
+    TableStat stat = new TableStat();
+    TableStat childStat;
+    long avgRows = 0, numBytes = 0, numRows = 0;
+    int numBlocks = 0, numPartitions = 0;
+    List<ColumnStat> columnStats = Lists.newArrayList();
+
+    Iterator<ExecutionBlock> it = unit.getBlock().getChildBlocks().iterator();
+    while (it.hasNext()) {
+      ExecutionBlock block = it.next();
+      SubQuery childSubQuery = unit.context.getSubQuery(block.getId());
+      childStat = childSubQuery.getTableStat();
+      avgRows += childStat.getAvgRows();
+      columnStats.addAll(childStat.getColumnStats());
+      numBlocks += childStat.getNumBlocks();
+      numBytes += childStat.getNumBytes();
+      numPartitions += childStat.getNumPartitions();
+      numRows += childStat.getNumRows();
+    }
+
+    stat.setColumnStats(columnStats);
+    stat.setNumBlocks(numBlocks);
+    stat.setNumBytes(numBytes);
+    stat.setNumPartitions(numPartitions);
+    stat.setNumRows(numRows);
+    stat.setAvgRows(avgRows);
+    return stat;
+  }
+
+  public TableMeta buildTableMeta() throws IOException {
+    finishTime = context.getClock().getTime();
+
+    TableStat stat;
+    if (block.hasUnion()) {
+      stat = computeStatFromUnionBlock(this);
+    } else {
+      stat = computeStatFromTasks();
+    }
+    TableMeta meta = writeStat(this, stat);
+    meta.setStat(stat);
+    setTableMeta(meta);
+    return meta;
+  }
+
+  private TableStat computeStatFromTasks() {
+    List<TableStat> stats = Lists.newArrayList();
+    for (QueryUnit unit : getQueryUnits()) {
+      stats.add(unit.getStats());
+    }
+    TableStat tableStat = StatisticsUtil.aggregateTableStat(stats);
+    return tableStat;
+  }
+
+  private TableMeta writeStat(SubQuery subQuery, TableStat stat)
+      throws IOException {
+    ExecutionBlock execBlock = subQuery.getBlock();
+    StoreTableNode storeTableNode = execBlock.getStoreTableNode();
+    TableMeta meta = toTableMeta(storeTableNode);
+    meta.setStat(stat);
+    sm.writeTableMeta(sm.getTablePath(execBlock.getOutputName()), meta);
+    return meta;
+  }
+
+  private static TableMeta toTableMeta(StoreTableNode store) {
+    if (store.hasOptions()) {
+      return CatalogUtil.newTableMeta(store.getOutSchema(),
+          store.getStorageType(), store.getOptions());
+    } else {
+      return CatalogUtil.newTableMeta(store.getOutSchema(),
+          store.getStorageType());
+    }
+  }
+
+  private void stopScheduler() {
+    // If there are launched TaskRunners, send the 'shouldDie' message to all r
+    // via received task requests.
+    if (taskScheduler != null) {
+      taskScheduler.stop();
+    }
+  }
+
+  private void releaseContainers() {
+    // If there are still live TaskRunners, try to kill the containers.
+    eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP ,getId(),
+        containers.values()));
+  }
+
+  private void finish() {
+    TableMeta meta = null;
+    try {
+      meta = buildTableMeta();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    setTableMeta(meta);
+    setFinishTime();
+    eventHandler.handle(new SubQuerySucceeEvent(getId(), meta));
+  }
+
+  @Override
+  public void handle(SubQueryEvent event) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType());
+    }
+
+    try {
+      writeLock.lock();
+      SubQueryState oldState = getState();
+      try {
+        getStateMachine().doTransition(event.getType(), event);
+      } catch (InvalidStateTransitonException e) {
+        LOG.error("Can't handle this event at current state", e);
+        eventHandler.handle(new SubQueryEvent(getId(),
+            SubQueryEventType.SQ_INTERNAL_ERROR));
+      }
+
+      // notify the eventhandler of state change
+      if (LOG.isDebugEnabled()) {
+        if (oldState != getState()) {
+          LOG.debug(getId() + " SubQuery Transitioned from " + oldState + " to "
+              + getState());
+        }
+      }
+    }
+
+    finally {
+      writeLock.unlock();
+    }
+  }
+
+  private static class InitAndRequestContainer implements MultipleArcTransition<SubQuery,
+      SubQueryEvent, SubQueryState> {
+
+    @Override
+    public SubQueryState transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      subQuery.setStartTime();
+      ExecutionBlock execBlock = subQuery.getBlock();
+      SubQueryState state;
+
+      try {
+        // Union operator does not require actual query processing. It is performed logically.
+        if (execBlock.hasUnion()) {
+          subQuery.finish();
+          state = SubQueryState.SUCCEEDED;
+        } else {
+          setRepartitionIfNecessary(subQuery);
+          createTasks(subQuery);
+
+          if (subQuery.tasks.size() == 0) { // if there is no tasks
+            subQuery.finish();
+            return SubQueryState.SUCCEEDED;
+          } else {
+            initTaskScheduler(subQuery);
+            allocateContainers(subQuery);
+            return SubQueryState.INIT;
+          }
+        }
+      } catch (Exception e) {
+        LOG.warn("SubQuery (" + subQuery.getId() + ") failed", e);
+        subQuery.eventHandler.handle(
+            new QueryDiagnosticsUpdateEvent(subQuery.getId().getQueryId(), e.getMessage()));
+        subQuery.eventHandler.handle(
+            new SubQueryCompletedEvent(subQuery.getId(), SubQueryState.FAILED));
+        return SubQueryState.FAILED;
+      }
+
+      return state;
+    }
+
+    private void initTaskScheduler(SubQuery subQuery) {
+      subQuery.taskScheduler = new TaskSchedulerImpl(subQuery.context);
+      subQuery.taskScheduler.init(subQuery.context.getConf());
+      subQuery.taskScheduler.start();
+    }
+
+    /**
+     * If a parent block requires a repartition operation, the method sets proper repartition
+     * methods and the number of partitions to a given subquery.
+     */
+    private static void setRepartitionIfNecessary(SubQuery subQuery) {
+      if (subQuery.getBlock().hasParentBlock()) {
+        int numTasks = calculatePartitionNum(subQuery);
+        Repartitioner.setPartitionNumberForTwoPhase(subQuery, numTasks);
+      }
+    }
+
+    /**
+     * Getting the desire number of partitions according to the volume of input data.
+     * This method is only used to determine the partition key number of hash join or aggregation.
+     *
+     * @param subQuery
+     * @return
+     */
+    public static int calculatePartitionNum(SubQuery subQuery) {
+      TajoConf conf = subQuery.context.getConf();
+      ExecutionBlock parent = subQuery.getBlock().getParentBlock();
+
+      GroupbyNode grpNode = null;
+      if (parent != null) {
+        grpNode = (GroupbyNode) PlannerUtil.findTopNode(
+            parent.getPlan(), ExprType.GROUP_BY);
+      }
+
+      // Is this subquery the first step of join?
+      if (parent != null && parent.getScanNodes().length == 2) {
+        Iterator<ExecutionBlock> child = parent.getChildBlocks().iterator();
+
+        // for inner
+        ExecutionBlock outer = child.next();
+        long outerVolume = getInputVolume(subQuery.context, outer);
+
+        // for inner
+        ExecutionBlock inner = child.next();
+        long innerVolume = getInputVolume(subQuery.context, inner);
+        LOG.info("Outer volume: " + Math.ceil((double)outerVolume / 1048576));
+        LOG.info("Inner volume: " + Math.ceil((double)innerVolume / 1048576));
+
+        long smaller = Math.min(outerVolume, innerVolume);
+
+        int mb = (int) Math.ceil((double)smaller / 1048576);
+        LOG.info("Smaller Table's volume is approximately " + mb + " MB");
+        // determine the number of task
+        int taskNum = (int) Math.ceil((double)mb /
+            conf.getIntVar(ConfVars.JOIN_PARTITION_VOLUME));
+        LOG.info("The determined number of join partitions is " + taskNum);
+        return taskNum;
+
+        // Is this subquery the first step of group-by?
+      } else if (grpNode != null) {
+
+        if (grpNode.getGroupingColumns().length == 0) {
+          return 1;
+        } else {
+          long volume = getInputVolume(subQuery.context, subQuery.block);
+
+          int mb = (int) Math.ceil((double)volume / 1048576);
+          LOG.info("Table's volume is approximately " + mb + " MB");
+          // determine the number of task
+          int taskNum = (int) Math.ceil((double)mb /
+              conf.getIntVar(ConfVars.AGGREGATION_PARTITION_VOLUME));
+          LOG.info("The determined number of aggregation partitions is " + taskNum);
+          return taskNum;
+        }
+      } else {
+        LOG.info("============>>>>> Unexpected Case! <<<<<================");
+        long volume = getInputVolume(subQuery.context, subQuery.block);
+
+        int mb = (int) Math.ceil((double)volume / 1048576);
+        LOG.info("Table's volume is approximately " + mb + " MB");
+        // determine the number of task per 128MB
+        int taskNum = (int) Math.ceil((double)mb / 128);
+        LOG.info("The determined number of partitions is " + taskNum);
+        return taskNum;
+      }
+    }
+
+    private static void createTasks(SubQuery subQuery) throws IOException {
+      ExecutionBlock execBlock = subQuery.getBlock();
+      QueryUnit [] tasks;
+      if (execBlock.isLeafBlock() && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
+        tasks = createLeafTasks(subQuery);
+
+      } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
+        tasks = Repartitioner.createJoinTasks(subQuery);
+
+      } else { // Case 3: Others (Sort or Aggregation)
+        int numTasks = getNonLeafTaskNum(subQuery);
+        SubQueryId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
+        SubQuery child = subQuery.context.getSubQuery(childId);
+        tasks = Repartitioner.createNonLeafTask(subQuery, child, numTasks);
+      }
+
+      LOG.info("Create " + tasks.length + " Tasks");
+
+      for (QueryUnit task : tasks) {
+        subQuery.addTask(task);
+      }
+    }
+
+    /**
+     * Getting the desire number of tasks according to the volume of input data
+     *
+     * @param subQuery
+     * @return
+     */
+    public static int getNonLeafTaskNum(SubQuery subQuery) {
+      // Getting intermediate data size
+      long volume = getInputVolume(subQuery.context, subQuery.getBlock());
+
+      int mb = (int) Math.ceil((double)volume / 1048576);
+      LOG.info("Table's volume is approximately " + mb + " MB");
+      // determine the number of task per 64MB
+      int maxTaskNum = (int) Math.ceil((double)mb / 64);
+      LOG.info("The determined number of non-leaf tasks is " + maxTaskNum);
+      return maxTaskNum;
+    }
+
+    public static long getInputVolume(QueryContext context, ExecutionBlock execBlock) {
+      Map<String, TableDesc> tableMap = context.getTableDescMap();
+      if (execBlock.isLeafBlock()) {
+        ScanNode outerScan = execBlock.getScanNodes()[0];
+        TableStat stat = tableMap.get(outerScan.getFromTable().getTableName()).getMeta().getStat();
+        return stat.getNumBytes();
+      } else {
+        long aggregatedVolume = 0;
+        for (ExecutionBlock childBlock : execBlock.getChildBlocks()) {
+          SubQuery subquery = context.getSubQuery(childBlock.getId());
+          aggregatedVolume += subquery.getTableStat().getNumBytes();
+        }
+
+        return aggregatedVolume;
+      }
+    }
+
+    public static void allocateContainers(SubQuery subQuery) {
+      ExecutionBlock execBlock = subQuery.getBlock();
+      QueryUnit [] tasks = subQuery.getQueryUnits();
+
+      int numClusterNodes = subQuery.getContext().getNumClusterNode();
+      int numRequest = 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);
+
+      Priority priority = Records.newRecord(Priority.class);
+      priority.setPriority(subQuery.getPriority());
+      ContainerAllocationEvent event =
+          new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ,
+              subQuery.getId(), priority, resource, numRequest,
+              execBlock.isLeafBlock(), 0.0f);
+      subQuery.eventHandler.handle(event);
+    }
+
+    private static QueryUnit [] createLeafTasks(SubQuery subQuery) throws IOException {
+      ExecutionBlock execBlock = subQuery.getBlock();
+      ScanNode[] scans = execBlock.getScanNodes();
+      Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");
+      TableMeta meta;
+      Path inputPath;
+
+      ScanNode scan = scans[0];
+      TableDesc desc = subQuery.context.getTableDescMap().get(scan.getFromTable().getTableName());
+      inputPath = desc.getPath();
+      meta = desc.getMeta();
+
+      // TODO - should be change the inner directory
+      Path oldPath = new Path(inputPath, "data");
+      FileSystem fs = inputPath.getFileSystem(subQuery.context.getConf());
+      if (fs.exists(oldPath)) {
+        inputPath = oldPath;
+      }
+      List<Fragment> fragments = subQuery.getStorageManager().getSplits(scan.getTableId(), meta, inputPath);
+
+      QueryUnit queryUnit;
+      List<QueryUnit> queryUnits = new ArrayList<QueryUnit>();
+
+      int i = 0;
+      for (Fragment fragment : fragments) {
+        queryUnit = newQueryUnit(subQuery, i++, fragment);
+        queryUnits.add(queryUnit);
+      }
+
+      return queryUnits.toArray(new QueryUnit[queryUnits.size()]);
+    }
+
+    private static QueryUnit newQueryUnit(SubQuery subQuery, int taskId, Fragment fragment) {
+      ExecutionBlock execBlock = subQuery.getBlock();
+      QueryUnit unit = new QueryUnit(
+          QueryIdFactory.newQueryUnitId(subQuery.getId(), taskId), execBlock.isLeafBlock(),
+          subQuery.eventHandler);
+      unit.setLogicalPlan(execBlock.getPlan());
+      unit.setFragment2(fragment);
+      return unit;
+    }
+  }
+
+  int i = 0;
+  private static class ContainerLaunchTransition
+      implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent event) {
+      SubQueryContainerAllocationEvent allocationEvent =
+          (SubQueryContainerAllocationEvent) event;
+      for (Container container : allocationEvent.getAllocatedContainer()) {
+        ContainerId cId = container.getId();
+        if (subQuery.containers.containsKey(cId)) {
+          LOG.info(">>>>>>>>>>>> Duplicate Container! <<<<<<<<<<<");
+        }
+        subQuery.containers.put(cId, container);
+        // TODO - This is debugging message. Should be removed
+        subQuery.i++;
+      }
+      LOG.info("SubQuery (" + subQuery.getId() + ") has " + subQuery.i + " containers!");
+      subQuery.eventHandler.handle(
+          new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
+              subQuery.getId(), allocationEvent.getAllocatedContainer()));
+
+      subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
+          SubQueryEventType.SQ_START));
+    }
+  }
+
+  private static class StartTransition implements
+      MultipleArcTransition<SubQuery, SubQueryEvent, SubQueryState> {
+
+    @Override
+    public SubQueryState transition(SubQuery subQuery,
+                           SubQueryEvent subQueryEvent) {
+      // schedule tasks
+      try {
+        for (QueryUnitId taskId : subQuery.tasks.keySet()) {
+          subQuery.eventHandler.handle(new TaskEvent(taskId, TaskEventType.T_SCHEDULE));
+        }
+
+        return  SubQueryState.RUNNING;
+      } catch (Exception e) {
+        LOG.warn("SubQuery (" + subQuery.getId() + ") failed", e);
+        return SubQueryState.FAILED;
+      }
+    }
+  }
+
+  private static class TaskCompletedTransition
+      implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery,
+                                     SubQueryEvent event) {
+      subQuery.completedTaskCount++;
+      SubQueryTaskEvent taskEvent = (SubQueryTaskEvent) event;
+      QueryUnitAttempt task = subQuery.getQueryUnit(taskEvent.getTaskId()).getSuccessfulAttempt();
+
+      LOG.info(subQuery.getId() + " SubQuery Succeeded " + subQuery.completedTaskCount + "/"
+          + subQuery.tasks.size() + " on " + task.getHost());
+      if (subQuery.completedTaskCount == subQuery.tasks.size()) {
+        subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
+            SubQueryEventType.SQ_SUBQUERY_COMPLETED));
+      }
+    }
+  }
+
+  private static class SubQueryCompleteTransition
+      implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery, SubQueryEvent subQueryEvent) {
+      // TODO - Commit subQuery & do cleanup
+      // TODO - records succeeded, failed, killed completed task
+      // TODO - records metrics
+      subQuery.stopScheduler();
+      subQuery.releaseContainers();
+      subQuery.finish();
+    }
+  }
+
+  private static class InternalErrorTransition
+      implements SingleArcTransition<SubQuery, SubQueryEvent> {
+
+    @Override
+    public void transition(SubQuery subQuery,
+                           SubQueryEvent subQueryEvent) {
+
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
new file mode 100644
index 0000000..c8256ec
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQueryState.java
@@ -0,0 +1,28 @@
+/**
+ * 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;
+
+public enum SubQueryState {
+  NEW,
+  CONTAINER_ALLOCATED,
+  INIT,
+  RUNNING,
+  SUCCEEDED,
+  FAILED
+}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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
index c615532..c01cabc 100644
--- 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
@@ -25,17 +25,18 @@ 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.*;
+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.TajoProtos.QueryState;
-import org.apache.tajo.master.QueryMaster.QueryContext;
-import org.apache.tajo.master.SubQueryState;
 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;
@@ -115,9 +116,10 @@ public class RMContainerAllocator extends AMRMClientImpl
             if (!stopped.get()) {
               LOG.warn("Allocated thread interrupted. Returning.");
             }
-            return;
+            break;
           }
         }
+        LOG.info("Allocated thread stopped");
       }
     });
     allocatorThread.setName("RMContainerAllocator");
@@ -126,24 +128,9 @@ public class RMContainerAllocator extends AMRMClientImpl
 
   public void stop() {
     stopped.set(true);
+    allocatorThread.interrupt();
+    LOG.info("RMContainerAllocator stopped");
     super.stop();
-    FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;
-    QueryState state = context.getQuery().getState();
-    if (state == QueryState.QUERY_SUCCEEDED) {
-      finishState = FinalApplicationStatus.SUCCEEDED;
-    } else if (state == QueryState.QUERY_KILLED
-        || (state == QueryState.QUERY_RUNNING)) {
-      finishState = FinalApplicationStatus.KILLED;
-    } else if (state == QueryState.QUERY_FAILED
-        || state == QueryState.QUERY_ERROR) {
-      finishState = FinalApplicationStatus.FAILED;
-    }
-
-    try {
-      unregisterApplicationMaster(finishState, "", "http://localhost:1234");
-    } catch (YarnRemoteException e) {
-      LOG.error(e);
-    }
   }
 
   private final Map<Priority, SubQueryId> subQueryMap =

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 9e281c6..a56841b 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,7 @@ 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.engine.MasterWorkerProtos.*;
+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,7 +45,7 @@ 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.MasterWorkerProtocol.MasterWorkerProtocolService.Interface;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.Interface;
 import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
 import org.apache.tajo.master.ExecutionBlock.PartitionType;
 import org.apache.tajo.rpc.NullCallback;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 ca8ef43..a41b280 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
@@ -37,11 +37,9 @@ 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.MasterWorkerProtos.QueryUnitRequestProto;
 import org.apache.tajo.engine.query.QueryUnitRequestImpl;
-import org.apache.tajo.ipc.MasterWorkerProtocol;
-import org.apache.tajo.ipc.MasterWorkerProtocol.MasterWorkerProtocolService;
-import org.apache.tajo.ipc.MasterWorkerProtocol.MasterWorkerProtocolService.Interface;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.*;
 import org.apache.tajo.rpc.CallFuture2;
 import org.apache.tajo.rpc.NullCallback;
 import org.apache.tajo.rpc.ProtoAsyncRpcClient;
@@ -53,7 +51,7 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Map;
 import java.util.concurrent.*;
 
-import static org.apache.tajo.engine.MasterWorkerProtos.TaskFatalErrorReport;
+import static org.apache.tajo.ipc.QueryMasterProtocol.*;
 
 /**
  * The driver class for Tajo QueryUnit processing.
@@ -72,7 +70,7 @@ public class TaskRunner extends AbstractService {
   private final ContainerId containerId;
 
   // Cluster Management
-  private MasterWorkerProtocolService.Interface master;
+  private QueryMasterProtocol.QueryMasterProtocolService.Interface master;
 
   // for temporal or intermediate files
   private FileSystem localFS;
@@ -186,7 +184,7 @@ public class TaskRunner extends AbstractService {
       return nodeId.toString();
     }
 
-    public MasterWorkerProtocolService.Interface getMaster() {
+    public QueryMasterProtocolService.Interface getMaster() {
       return master;
     }
 
@@ -223,7 +221,7 @@ public class TaskRunner extends AbstractService {
     }
   }
 
-  static void fatalError(MasterWorkerProtocolService.Interface proxy,
+  static void fatalError(QueryMasterProtocolService.Interface proxy,
                          QueryUnitAttemptId taskAttemptId, String message) {
     TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder()
         .setId(taskAttemptId.getProto())
@@ -338,11 +336,11 @@ public class TaskRunner extends AbstractService {
   /**
    * TaskRunner takes 5 arguments as follows:
    * <ol>
-   * <li>1st: TaskRunnerListener hostname</li>
-   * <li>2nd: TaskRunnerListener port</li>
-   * <li>3nd: SubQueryId</li>
-   * <li>4th: NodeId</li>
-   * <li>5th: ContainerId</li>
+   * <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 {
@@ -356,17 +354,17 @@ public class TaskRunner extends AbstractService {
 
     UserGroupInformation.setConfiguration(conf);
 
-    // TaskRunnerListener's address
-    String host = args[0];
-    int port = Integer.parseInt(args[1]);
-    final InetSocketAddress masterAddr =
-        NetUtils.createSocketAddrForHost(host, port);
-
     // SubQueryId from String
-    final SubQueryId subQueryId = TajoIdUtils.newSubQueryId(args[2]);
+    final SubQueryId subQueryId = TajoIdUtils.newSubQueryId(args[0]);
     // NodeId has a form of hostname:port.
-    NodeId nodeId = ConverterUtils.toNodeId(args[3]);
-    ContainerId containerId = ConverterUtils.toContainerId(args[4]);
+    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
@@ -374,26 +372,29 @@ public class TaskRunner extends AbstractService {
         UserGroupInformation.createRemoteUser(conf.getVar(ConfVars.QUERY_USERNAME));
     //taskOwner.addToken(token);
 
-    // TaskRunnerListener RPC
+    // QueryMasterService RPC
     ProtoAsyncRpcClient client;
-    MasterWorkerProtocolService.Interface master;
+    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(MasterWorkerProtocol.class, masterAddr);
+            return new ProtoAsyncRpcClient(QueryMasterProtocol.class, masterAddr);
           }
         });
     master = client.getStub();
 
 
     TaskRunner taskRunner = new TaskRunner(subQueryId, nodeId, taskOwner, master, containerId);
-    taskRunner.init(conf);
-    taskRunner.start();
-    client.close();
-    LOG.info("TaskRunner (" + nodeId + ") main thread exiting");
-    System.exit(0);
+    try {
+      taskRunner.init(conf);
+      taskRunner.start();
+    } finally {
+      client.close();
+      LOG.info("TaskRunner (" + nodeId + ") main thread exiting");
+      System.exit(0);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
index c171c2b..6164553 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/CatalogProtos.proto
@@ -25,9 +25,9 @@ option java_generate_equals_and_hash = true;
 import "DataTypes.proto";
 
 enum StoreType {
-  MEM = 0;
-  CSV = 1;
-  RAW = 2;
+	MEM = 0;
+	CSV = 1;
+	RAW = 2;
   RCFILE = 3;
   ROWFILE = 4;
   HCFILE = 5;
@@ -35,147 +35,147 @@ enum StoreType {
 }
 
 enum OrderType {
-  ORDER_NONE = 0;
-  ASC = 1;
-  DSC = 2;
+    ORDER_NONE = 0;
+    ASC = 1;
+    DSC = 2;
 }
 
 enum CompressType {
-  COMP_NONE = 0;
-  NULL_SUPPRESS = 1;
-  RUN_LENGTH = 2;
-  BIT_VECTOR = 3;
-  DICTIONARY = 4;
-  SNAPPY = 5;
-  LZ = 6;
+    COMP_NONE = 0;
+    NULL_SUPPRESS = 1;
+    RUN_LENGTH = 2;
+    BIT_VECTOR = 3;
+    DICTIONARY = 4;
+    SNAPPY = 5;
+    LZ = 6;
 }
 
 message ColumnMetaProto {
-  required DataType dataType = 1;
-  required bool compressed = 2;
-  required bool sorted = 3;
-  required bool contiguous = 4;
-  required StoreType storeType = 5;
-  required CompressType compType = 6;
-  required int64 startRid = 7;
-  required int32 recordNum = 8;
-  required int32 offsetToIndex = 9;
+    required DataType dataType = 1;
+    required bool compressed = 2;
+    required bool sorted = 3;
+    required bool contiguous = 4;
+    required StoreType storeType = 5;
+    required CompressType compType = 6;
+    required int64 startRid = 7;
+    required int32 recordNum = 8;
+    required int32 offsetToIndex = 9;
 }
 
 message ColumnProto {
-  required string columnName = 1;
-  required DataType dataType = 2;
+	required string columnName = 1;
+	required DataType dataType = 2;
 }
 
 message SchemaProto {
-  repeated ColumnProto fields = 1;
+	repeated ColumnProto fields = 1;
 }
 
 message KeyValueProto {
-  required string key = 1;
-  required string value = 2;
+	required string key = 1;
+	required string value = 2;
 }
 
 message KeyValueSetProto {
-  repeated KeyValueProto keyval = 1;
+	repeated KeyValueProto keyval = 1;
 }
 
 message FragmentProto {
-  required string id = 1;
-  required string path = 2;
-  required int64 startOffset = 3;
-  required int64 length = 4;
-  required TableProto meta = 5;
-  optional TableStatProto stat = 6;
+	required string id = 1;
+	required string path = 2;
+	required int64 startOffset = 3;
+	required int64 length = 4;
+	required TableProto meta = 5;
+	optional TableStatProto stat = 6;
   optional bool distCached = 7 [default = false];
 }
 
 message TableProto {
-  required SchemaProto schema = 1;
-  required StoreType storeType = 2;
-  required KeyValueSetProto params = 3;
-  optional TableStatProto stat = 4;
+    required SchemaProto schema = 1;
+    required StoreType storeType = 2;
+    required KeyValueSetProto params = 3;
+    optional TableStatProto stat = 4;
 }
 
 message TableDescProto {
-  required string id = 1;
-  required string path = 2;
-  required TableProto meta = 3;
+	required string id = 1;
+	required string path = 2;
+	required TableProto meta = 3;
 }
 
 enum FunctionType {
-  GENERAL = 0;
-  AGGREGATION = 1;
+	GENERAL = 0;
+	AGGREGATION = 1;
 }
 
 message FunctionDescProto {
-  required string signature = 1;
-  required string className = 2;
-  required FunctionType type = 3;
-  repeated DataType parameterTypes = 4;
-  required DataType returnType = 5;
+	required string signature = 1;
+	required string className = 2;
+	required FunctionType type = 3;
+	repeated DataType parameterTypes = 4;
+	required DataType returnType = 5;
 }
 
 message IndexDescProto {
-  required string name = 1;
-  required string tableId = 2;
-  required ColumnProto column = 3;
-  required IndexMethod indexMethod = 4;
-  optional bool isUnique = 5 [default = false];
-  optional bool isClustered = 6 [default = false];
-  optional bool isAscending = 7 [default = false];
+    required string name = 1;
+    required string tableId = 2;
+    required ColumnProto column = 3;
+    required IndexMethod indexMethod = 4;
+    optional bool isUnique = 5 [default = false];
+    optional bool isClustered = 6 [default = false];
+    optional bool isAscending = 7 [default = false];
 }
 
 enum IndexMethod {
-  TWO_LEVEL_BIN_TREE = 0;
-  BTREE = 1;
-  HASH = 2;
-  BITMAP = 3;
+    TWO_LEVEL_BIN_TREE = 0;
+    BTREE = 1;
+    HASH = 2;
+    BITMAP = 3;
 }
 
 message GetAllTableNamesResponse {
-  repeated string tableName = 1;
+    repeated string tableName = 1;
 }
 
 message GetIndexRequest {
-  required string tableName = 1;
-  required string columnName = 2;
+    required string tableName = 1;
+    required string columnName = 2;
 }
 
 message GetFunctionsResponse {
-  repeated FunctionDescProto functionDesc = 1;
+	repeated FunctionDescProto functionDesc = 1;
 }
 
 message UnregisterFunctionRequest {
-  required string signature = 1;
-  repeated DataType parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message GetFunctionMetaRequest {
-  required string signature = 1;
-  repeated DataType parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message ContainFunctionRequest {
-  required string signature = 1;
-  repeated DataType parameterTypes = 2;
+	required string signature = 1;
+	repeated DataType parameterTypes = 2;
 }
 
 message TableStatProto {
-  required int64 numRows = 1;
-  required int64 numBytes = 2;
-  optional int32 numBlocks = 3;
-  optional int32 numPartitions = 4;
-  optional int64 avgRows = 5;
-  repeated ColumnStatProto colStat = 6;
+	required int64 numRows = 1;
+	required int64 numBytes = 2;
+	optional int32 numBlocks = 3;
+	optional int32 numPartitions = 4;
+	optional int64 avgRows = 5;
+	repeated ColumnStatProto colStat = 6;
 }
 
 message ColumnStatProto {
-  required ColumnProto column = 1;
-  optional int64 numDistVal = 2;
-  optional int64 numNulls = 3;
-  optional bytes minValue = 4;
-  optional bytes maxValue = 5;
+    required ColumnProto column = 1;
+    optional int64 numDistVal = 2;
+    optional int64 numNulls = 3;
+    optional bytes minValue = 4;
+    optional bytes maxValue = 5;
 }
 
 enum StatType {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/9d020883/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 cbcccd3..61c14c4 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
@@ -49,7 +49,7 @@ message UpdateQueryResponse {
   optional string errorMessage = 2;
 }
 
-message SubmitQueryRespose {
+message SubmitQueryResponse {
   required ResultCode resultCode = 1;
   optional ApplicationAttemptIdProto queryId = 2;
   optional string errorMessage = 3;
@@ -94,6 +94,8 @@ message GetQueryStatusResponse {
   optional int64 finishTime = 7;
   optional bool hasResult = 8;
   optional string errorMessage = 9;
+  optional string queryMasterHost = 10;
+  optional int32 queryMasterPort = 11;
 }
 
 message GetClusterInfoRequest {
@@ -135,7 +137,7 @@ message TableResponse {
 
 service ClientProtocolService {
   rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
-  rpc submitQuery(QueryRequest) returns (SubmitQueryRespose);
+  rpc submitQuery(QueryRequest) returns (SubmitQueryResponse);
   rpc updateQuery(QueryRequest) returns (UpdateQueryResponse);
   rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
   rpc getQueryList(GetQueryListRequest) returns (GetQueryListResponse);
@@ -151,7 +153,6 @@ service ClientProtocolService {
   rpc detachTable(StringProto) returns (BoolProto);
 
 
-
   // TODO - to be implemented
   //
   // authenticate