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