You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2013/08/26 14:29:10 UTC
[1/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Updated Branches:
refs/heads/master 8b8b6683d -> d48f2667b
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml b/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
index c003637..20b80aa 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
+++ b/tajo-core/tajo-core-backend/src/test/resources/tajo-default.xml
@@ -35,4 +35,10 @@
<name>tajo.task.localdir</name>
<value>/tmp/tajo-localdir</value>
</property>
+
+ <property>
+ <name>tajo.resource.manager</name>
+ <value>org.apache.tajo.master.rm.TajoWorkerResourceManager</value>
+ <description>org.apache.tajo.master.rm.TajoWorkerResourceManager or org.apache.tajo.master.rm.YarnTajoResourceManager</description>
+ </property>
</configuration>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/pom.xml b/tajo-core/tajo-core-pullserver/pom.xml
index 1ce6aba..b46caba 100644
--- a/tajo-core/tajo-core-pullserver/pom.xml
+++ b/tajo-core/tajo-core-pullserver/pom.xml
@@ -63,6 +63,7 @@
<artifactId>netty</artifactId>
<scope>compile</scope>
</dependency>
+
<dependency>
<groupId>commons-logging</groupId>
<artifactId>commons-logging</artifactId>
@@ -154,16 +155,6 @@
<reporting>
<plugins>
<plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>findbugs-maven-plugin</artifactId>
- <version>2.3.2</version>
- <configuration>
- <xmlOutput>true</xmlOutput>
- <xmlOutputDirectory>target/findbugs</xmlOutputDirectory>
- <!--<excludeFilterFile>src/main/findbugs/findbugs-exclude.xml</excludeFilterFile>-->
- </configuration>
- </plugin>
- <plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-project-info-reports-plugin</artifactId>
<version>2.4</version>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
index 1aae489..31db15c 100644
--- a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/HttpDataServerHandler.java
@@ -22,6 +22,9 @@ import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.pullserver.retriever.DataRetriever;
+import org.apache.tajo.pullserver.retriever.FileChunk;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.channel.*;
import org.jboss.netty.handler.codec.frame.TooLongFrameException;
@@ -29,9 +32,6 @@ import org.jboss.netty.handler.codec.http.*;
import org.jboss.netty.handler.ssl.SslHandler;
import org.jboss.netty.handler.stream.ChunkedFile;
import org.jboss.netty.util.CharsetUtil;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.pullserver.retriever.DataRetriever;
-import org.apache.tajo.pullserver.retriever.FileChunk;
import java.io.*;
import java.net.URLDecoder;
@@ -51,7 +51,8 @@ import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
public class HttpDataServerHandler extends SimpleChannelUpstreamHandler {
private final static Log LOG = LogFactory.getLog(HttpDataServerHandler.class);
- Map<SubQueryId, DataRetriever> retrievers = new ConcurrentHashMap<SubQueryId, DataRetriever>();
+ Map<ExecutionBlockId, DataRetriever> retrievers =
+ new ConcurrentHashMap<ExecutionBlockId, DataRetriever>();
private String userName;
private String appId;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
index 0bdf504..924289e 100644
--- a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/PullServerAuxService.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.security.ssl.SSLFactory;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices;
import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
@@ -51,6 +52,7 @@ import org.apache.tajo.storage.Tuple;
import org.apache.tajo.storage.TupleComparator;
import org.apache.tajo.storage.TupleRange;
import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.apache.tajo.util.TajoIdUtils;
import org.jboss.netty.bootstrap.ServerBootstrap;
import org.jboss.netty.buffer.ChannelBuffers;
import org.jboss.netty.channel.*;
@@ -104,6 +106,7 @@ public class PullServerAuxService extends AbstractService
private int sslFileBufferSize;
private ApplicationId appId;
+ private QueryId queryId;
private FileSystem localFS;
/**
@@ -189,6 +192,7 @@ public class PullServerAuxService extends AbstractService
// TODO these bytes should be versioned
// TODO: Once SHuffle is out of NM, this can use MR APIs
this.appId = appId;
+ this.queryId = TajoIdUtils.parseQueryId(appId.toString());
this.userName = user;
userRsrc.put(appId.toString(), user);
}
@@ -250,6 +254,10 @@ public class PullServerAuxService extends AbstractService
DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
}
+ public int getPort() {
+ return port;
+ }
+
@Override
public synchronized void stop() {
try {
@@ -384,7 +392,12 @@ public class PullServerAuxService extends AbstractService
List<String> taskIds = splitMaps(taskIdList);
// the working dir of tajo worker for each query
- String queryBaseDir = appId + "/output" + "/";
+ String queryBaseDir = queryId + "/output" + "/";
+
+ LOG.info("PullServer request param: repartitionType=" + repartitionType +
+ ", sid=" + sid + ", partitionId=" + partitionId + ", taskIds=" + taskIdList);
+
+ LOG.info("PullServer baseDir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname) + "/" + queryBaseDir);
// if a subquery requires a range partitioning
if (repartitionType.equals("r")) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
new file mode 100644
index 0000000..3df3672
--- /dev/null
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/TajoPullServerService.java
@@ -0,0 +1,652 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.pullserver;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.pullserver.retriever.FileChunk;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.TupleComparator;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.index.bst.BSTIndex;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.*;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.*;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.setContentLength;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.*;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+public class TajoPullServerService extends AbstractService {
+
+ private static final Log LOG = LogFactory.getLog(TajoPullServerService.class);
+
+ public static final String SHUFFLE_MANAGE_OS_CACHE = "tajo.pullserver.manage.os.cache";
+ public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+ public static final String SHUFFLE_READAHEAD_BYTES = "tajo.pullserver.readahead.bytes";
+ public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+ private int port;
+ private ChannelFactory selector;
+ private final ChannelGroup accepted = new DefaultChannelGroup();
+ private HttpPipelineFactory pipelineFact;
+ private int sslFileBufferSize;
+
+ private ApplicationId appId;
+ private FileSystem localFS;
+
+ /**
+ * Should the shuffle use posix_fadvise calls to manage the OS cache during
+ * sendfile
+ */
+ private boolean manageOsCache;
+ private int readaheadLength;
+ private ReadaheadPool readaheadPool = ReadaheadPool.getInstance();
+
+
+ public static final String PULLSERVER_SERVICEID = "tajo.pullserver";
+
+ private static final Map<String,String> userRsrc =
+ new ConcurrentHashMap<String,String>();
+ private static String userName;
+
+ public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+ "tajo.pullserver.ssl.file.buffer.size";
+
+ public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+ @Metrics(name="PullServerShuffleMetrics", about="PullServer output metrics", context="tajo")
+ static class ShuffleMetrics implements ChannelFutureListener {
+ @Metric({"OutputBytes","PullServer output in bytes"})
+ MutableCounterLong shuffleOutputBytes;
+ @Metric({"Failed","# of failed shuffle outputs"})
+ MutableCounterInt shuffleOutputsFailed;
+ @Metric({"Succeeded","# of succeeded shuffle outputs"})
+ MutableCounterInt shuffleOutputsOK;
+ @Metric({"Connections","# of current shuffle connections"})
+ MutableGaugeInt shuffleConnections;
+
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ if (future.isSuccess()) {
+ shuffleOutputsOK.incr();
+ } else {
+ shuffleOutputsFailed.incr();
+ }
+ shuffleConnections.decr();
+ }
+ }
+
+ final ShuffleMetrics metrics;
+
+ TajoPullServerService(MetricsSystem ms) {
+ super("httpshuffle");
+ metrics = ms.register(new ShuffleMetrics());
+ }
+
+ @SuppressWarnings("UnusedDeclaration")
+ public TajoPullServerService() {
+ this(DefaultMetricsSystem.instance());
+ }
+
+ /**
+ * Serialize the shuffle port into a ByteBuffer for use later on.
+ * @param port the port to be sent to the ApplciationMaster
+ * @return the serialized form of the port.
+ */
+ public static ByteBuffer serializeMetaData(int port) throws IOException {
+ //TODO these bytes should be versioned
+ DataOutputBuffer port_dob = new DataOutputBuffer();
+ port_dob.writeInt(port);
+ return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+ }
+
+ /**
+ * A helper function to deserialize the metadata returned by PullServerAuxService.
+ * @param meta the metadata returned by the PullServerAuxService
+ * @return the port the PullServer Handler is listening on to serve shuffle data.
+ */
+ public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+ //TODO this should be returning a class not just an int
+ DataInputByteBuffer in = new DataInputByteBuffer();
+ in.reset(meta);
+ return in.readInt();
+ }
+
+ public void initApp(String user, ApplicationId appId, ByteBuffer secret) {
+ // TODO these bytes should be versioned
+ // TODO: Once SHuffle is out of NM, this can use MR APIs
+ this.appId = appId;
+ this.userName = user;
+ userRsrc.put(appId.toString(), user);
+ }
+
+ public void stopApp(ApplicationId appId) {
+ userRsrc.remove(appId.toString());
+ }
+
+ @Override
+ public synchronized void init(Configuration conf) {
+ try {
+ manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
+ DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
+
+ readaheadLength = conf.getInt(SHUFFLE_READAHEAD_BYTES,
+ DEFAULT_SHUFFLE_READAHEAD_BYTES);
+
+ ThreadFactory bossFactory = new ThreadFactoryBuilder()
+ .setNameFormat("PullServerAuxService Netty Boss #%d")
+ .build();
+ ThreadFactory workerFactory = new ThreadFactoryBuilder()
+ .setNameFormat("PullServerAuxService Netty Worker #%d")
+ .build();
+
+ selector = new NioServerSocketChannelFactory(
+ Executors.newCachedThreadPool(bossFactory),
+ Executors.newCachedThreadPool(workerFactory));
+
+ localFS = new LocalFileSystem();
+ super.init(new Configuration(conf));
+ } catch (Throwable t) {
+ LOG.error(t);
+ }
+ }
+
+ // TODO change AbstractService to throw InterruptedException
+ @Override
+ public synchronized void start() {
+ Configuration conf = getConfig();
+ ServerBootstrap bootstrap = new ServerBootstrap(selector);
+ try {
+ pipelineFact = new HttpPipelineFactory(conf);
+ } catch (Exception ex) {
+ throw new RuntimeException(ex);
+ }
+ bootstrap.setPipelineFactory(pipelineFact);
+ port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+ ConfVars.PULLSERVER_PORT.defaultIntVal);
+ Channel ch = bootstrap.bind(new InetSocketAddress(port));
+ accepted.add(ch);
+ port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+ conf.set(ConfVars.PULLSERVER_PORT.varname, Integer.toString(port));
+ pipelineFact.PullServer.setPort(port);
+ LOG.info(getName() + " listening on port " + port);
+ super.start();
+
+ sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+ DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+ }
+
+ public int getPort() {
+ return port;
+ }
+
+ @Override
+ public synchronized void stop() {
+ try {
+ accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+ ServerBootstrap bootstrap = new ServerBootstrap(selector);
+ bootstrap.releaseExternalResources();
+ pipelineFact.destroy();
+
+ localFS.close();
+ } catch (Throwable t) {
+ LOG.error(t);
+ } finally {
+ super.stop();
+ }
+ }
+
+ public synchronized ByteBuffer getMeta() {
+ try {
+ return serializeMetaData(port);
+ } catch (IOException e) {
+ LOG.error("Error during getMeta", e);
+ // TODO add API to AuxiliaryServices to report failures
+ return null;
+ }
+ }
+
+ class HttpPipelineFactory implements ChannelPipelineFactory {
+
+ final PullServer PullServer;
+ private SSLFactory sslFactory;
+
+ public HttpPipelineFactory(Configuration conf) throws Exception {
+ PullServer = new PullServer(conf);
+ if (conf.getBoolean(ConfVars.SHUFFLE_SSL_ENABLED_KEY.varname,
+ ConfVars.SHUFFLE_SSL_ENABLED_KEY.defaultBoolVal)) {
+ sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+ sslFactory.init();
+ }
+ }
+
+ public void destroy() {
+ if (sslFactory != null) {
+ sslFactory.destroy();
+ }
+ }
+
+ @Override
+ public ChannelPipeline getPipeline() throws Exception {
+ ChannelPipeline pipeline = Channels.pipeline();
+ if (sslFactory != null) {
+ pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+ }
+ pipeline.addLast("decoder", new HttpRequestDecoder());
+ pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+ pipeline.addLast("encoder", new HttpResponseEncoder());
+ pipeline.addLast("chunking", new ChunkedWriteHandler());
+ pipeline.addLast("shuffle", PullServer);
+ return pipeline;
+ // TODO factor security manager into pipeline
+ // TODO factor out encode/decode to permit binary shuffle
+ // TODO factor out decode of index to permit alt. models
+ }
+ }
+
+ class PullServer extends SimpleChannelUpstreamHandler {
+
+ private final Configuration conf;
+// private final IndexCache indexCache;
+ private final LocalDirAllocator lDirAlloc =
+ new LocalDirAllocator(ConfVars.TASK_LOCAL_DIR.varname);
+ private int port;
+
+ public PullServer(Configuration conf) {
+ this.conf = conf;
+// indexCache = new IndexCache(new JobConf(conf));
+ this.port = conf.getInt(ConfVars.PULLSERVER_PORT.varname,
+ ConfVars.PULLSERVER_PORT.defaultIntVal);
+ }
+
+ public void setPort(int port) {
+ this.port = port;
+ }
+
+ private List<String> splitMaps(List<String> mapq) {
+ if (null == mapq) {
+ return null;
+ }
+ final List<String> ret = new ArrayList<String>();
+ for (String s : mapq) {
+ Collections.addAll(ret, s.split(","));
+ }
+ return ret;
+ }
+
+ @Override
+ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
+ throws Exception {
+
+ HttpRequest request = (HttpRequest) e.getMessage();
+ if (request.getMethod() != GET) {
+ sendError(ctx, METHOD_NOT_ALLOWED);
+ return;
+ }
+
+ // Parsing the URL into key-values
+ final Map<String, List<String>> params =
+ new QueryStringDecoder(request.getUri()).getParameters();
+ final List<String> types = params.get("type");
+ final List<String> qids = params.get("qid");
+ final List<String> taskIdList = params.get("ta");
+ final List<String> subQueryIds = params.get("sid");
+ final List<String> partitionIds = params.get("p");
+
+ if (types == null || taskIdList == null || subQueryIds == null || qids == null
+ || partitionIds == null) {
+ sendError(ctx, "Required queryId, type, taskIds, subquery Id, and partition id",
+ BAD_REQUEST);
+ return;
+ }
+
+ if (qids.size() != 1 && types.size() != 1 || subQueryIds.size() != 1) {
+ sendError(ctx, "Required qids, type, taskIds, subquery Id, and partition id",
+ BAD_REQUEST);
+ return;
+ }
+
+ final List<FileChunk> chunks = Lists.newArrayList();
+
+ String queryId = qids.get(0);
+ String repartitionType = types.get(0);
+ String sid = subQueryIds.get(0);
+ String partitionId = partitionIds.get(0);
+ List<String> taskIds = splitMaps(taskIdList);
+
+ LOG.info("PullServer request param: repartitionType=" + repartitionType +
+ ", sid=" + sid + ", partitionId=" + partitionId + ", taskIds=" + taskIdList);
+
+ // the working dir of tajo worker for each query
+ String queryBaseDir = queryId.toString() + "/output";
+
+ LOG.info("PullServer baseDir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname) + "/" + queryBaseDir);
+
+ // if a subquery requires a range partitioning
+ if (repartitionType.equals("r")) {
+ String ta = taskIds.get(0);
+ Path path = localFS.makeQualified(
+ lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" + ta + "/output/", conf));
+
+ String startKey = params.get("start").get(0);
+ String endKey = params.get("end").get(0);
+ boolean last = params.get("final") != null;
+
+ FileChunk chunk;
+ try {
+ chunk = getFileCunks(path, startKey, endKey, last);
+ } catch (Throwable t) {
+ LOG.error("ERROR Request: " + request.getUri(), t);
+ sendError(ctx, "Cannot get file chunks to be sent", BAD_REQUEST);
+ return;
+ }
+ if (chunk != null) {
+ chunks.add(chunk);
+ }
+
+ // if a subquery requires a hash repartition
+ } else if (repartitionType.equals("h")) {
+ for (String ta : taskIds) {
+ Path path = localFS.makeQualified(
+ lDirAlloc.getLocalPathToRead(queryBaseDir + "/" + sid + "/" +
+ ta + "/output/" + partitionId, conf));
+ File file = new File(path.toUri());
+ FileChunk chunk = new FileChunk(file, 0, file.length());
+ chunks.add(chunk);
+ }
+ } else {
+ LOG.error("Unknown repartition type: " + repartitionType);
+ return;
+ }
+
+ // Write the content.
+ Channel ch = e.getChannel();
+ if (chunks.size() == 0) {
+ HttpResponse response = new DefaultHttpResponse(HTTP_1_1, NO_CONTENT);
+ ch.write(response);
+ if (!isKeepAlive(request)) {
+ ch.close();
+ }
+ } else {
+ FileChunk[] file = chunks.toArray(new FileChunk[chunks.size()]);
+ HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+ long totalSize = 0;
+ for (FileChunk chunk : file) {
+ totalSize += chunk.length();
+ }
+ setContentLength(response, totalSize);
+
+ // Write the initial line and the header.
+ ch.write(response);
+
+ ChannelFuture writeFuture = null;
+
+ for (FileChunk chunk : file) {
+ writeFuture = sendFile(ctx, ch, chunk);
+ if (writeFuture == null) {
+ sendError(ctx, NOT_FOUND);
+ return;
+ }
+ }
+
+ // Decide whether to close the connection or not.
+ if (!isKeepAlive(request)) {
+ // Close the connection when the whole content is written out.
+ writeFuture.addListener(ChannelFutureListener.CLOSE);
+ }
+ }
+ }
+
+ private ChannelFuture sendFile(ChannelHandlerContext ctx,
+ Channel ch,
+ FileChunk file) throws IOException {
+ RandomAccessFile spill;
+ try {
+ spill = new RandomAccessFile(file.getFile(), "r");
+ } catch (FileNotFoundException e) {
+ LOG.info(file.getFile() + " not found");
+ return null;
+ }
+ ChannelFuture writeFuture;
+ if (ch.getPipeline().get(SslHandler.class) == null) {
+ final FadvisedFileRegion partition = new FadvisedFileRegion(spill,
+ file.startOffset, file.length(), manageOsCache, readaheadLength,
+ readaheadPool, file.getFile().getAbsolutePath());
+ writeFuture = ch.write(partition);
+ writeFuture.addListener(new ChannelFutureListener() {
+ // TODO error handling; distinguish IO/connection failures,
+ // attribute to appropriate spill output
+ @Override
+ public void operationComplete(ChannelFuture future) {
+ partition.releaseExternalResources();
+ }
+ });
+ } else {
+ // HTTPS cannot be done with zero copy.
+ final FadvisedChunkedFile chunk = new FadvisedChunkedFile(spill,
+ file.startOffset, file.length, sslFileBufferSize,
+ manageOsCache, readaheadLength, readaheadPool,
+ file.getFile().getAbsolutePath());
+ writeFuture = ch.write(chunk);
+ }
+ metrics.shuffleConnections.incr();
+ metrics.shuffleOutputBytes.incr(file.length); // optimistic
+ return writeFuture;
+ }
+
+ private void sendError(ChannelHandlerContext ctx,
+ HttpResponseStatus status) {
+ sendError(ctx, "", status);
+ }
+
+ private void sendError(ChannelHandlerContext ctx, String message,
+ HttpResponseStatus status) {
+ HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+ response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+ response.setContent(
+ ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+
+ // Close the connection as soon as the error message is sent.
+ ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+ }
+
+ @Override
+ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+ throws Exception {
+ Channel ch = e.getChannel();
+ Throwable cause = e.getCause();
+ if (cause instanceof TooLongFrameException) {
+ sendError(ctx, BAD_REQUEST);
+ return;
+ }
+
+ LOG.error("PullServer error: ", cause);
+ if (ch.isConnected()) {
+ LOG.error("PullServer error " + e);
+ sendError(ctx, INTERNAL_SERVER_ERROR);
+ }
+ }
+ }
+
+ public FileChunk getFileCunks(Path outDir,
+ String startKey,
+ String endKey,
+ boolean last) throws IOException {
+ BSTIndex index = new BSTIndex(new TajoConf());
+ BSTIndex.BSTIndexReader idxReader =
+ index.getIndexReader(new Path(outDir, "index"));
+ idxReader.open();
+ Schema keySchema = idxReader.getKeySchema();
+ TupleComparator comparator = idxReader.getComparator();
+
+ LOG.info("BSTIndex is loaded from disk (" + idxReader.getFirstKey() + ", "
+ + idxReader.getLastKey());
+
+ File data = new File(URI.create(outDir.toUri() + "/output"));
+ byte [] startBytes = Base64.decodeBase64(startKey);
+ byte [] endBytes = Base64.decodeBase64(endKey);
+
+ Tuple start;
+ Tuple end;
+ try {
+ start = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, startBytes);
+ } catch (Throwable t) {
+ throw new IllegalArgumentException("StartKey: " + startKey
+ + ", decoded byte size: " + startBytes.length, t);
+ }
+
+ try {
+ end = RowStoreUtil.RowStoreDecoder.toTuple(keySchema, endBytes);
+ } catch (Throwable t) {
+ throw new IllegalArgumentException("EndKey: " + endKey
+ + ", decoded byte size: " + endBytes.length, t);
+ }
+
+
+ if(!comparator.isAscendingFirstKey()) {
+ Tuple tmpKey = start;
+ start = end;
+ end = tmpKey;
+ }
+
+ LOG.info("GET Request for " + data.getAbsolutePath() + " (start="+start+", end="+ end +
+ (last ? ", last=true" : "") + ")");
+
+ if (idxReader.getFirstKey() == null && idxReader.getLastKey() == null) { // if # of rows is zero
+ LOG.info("There is no contents");
+ return null;
+ }
+
+ if (comparator.compare(end, idxReader.getFirstKey()) < 0 ||
+ comparator.compare(idxReader.getLastKey(), start) < 0) {
+ LOG.info("Out of Scope (indexed data [" + idxReader.getFirstKey() + ", " + idxReader.getLastKey() +
+ "], but request start:" + start + ", end: " + end);
+ return null;
+ }
+
+ long startOffset;
+ long endOffset;
+ try {
+ startOffset = idxReader.find(start);
+ } catch (IOException ioe) {
+ LOG.error("State Dump (the requested range: "
+ + new TupleRange(keySchema, start, end) + ", idx min: "
+ + idxReader.getFirstKey() + ", idx max: "
+ + idxReader.getLastKey());
+ throw ioe;
+ }
+ try {
+ endOffset = idxReader.find(end);
+ if (endOffset == -1) {
+ endOffset = idxReader.find(end, true);
+ }
+ } catch (IOException ioe) {
+ LOG.error("State Dump (the requested range: "
+ + new TupleRange(keySchema, start, end) + ", idx min: "
+ + idxReader.getFirstKey() + ", idx max: "
+ + idxReader.getLastKey());
+ throw ioe;
+ }
+
+ // if startOffset == -1 then case 2-1 or case 3
+ if (startOffset == -1) { // this is a hack
+ // if case 2-1 or case 3
+ try {
+ startOffset = idxReader.find(start, true);
+ } catch (IOException ioe) {
+ LOG.error("State Dump (the requested range: "
+ + new TupleRange(keySchema, start, end) + ", idx min: "
+ + idxReader.getFirstKey() + ", idx max: "
+ + idxReader.getLastKey());
+ throw ioe;
+ }
+ }
+
+ if (startOffset == -1) {
+ throw new IllegalStateException("startOffset " + startOffset + " is negative \n" +
+ "State Dump (the requested range: "
+ + new TupleRange(keySchema, start, end) + ", idx min: " + idxReader.getFirstKey() + ", idx max: "
+ + idxReader.getLastKey());
+ }
+
+ // if greater than indexed values
+ if (last || (endOffset == -1
+ && comparator.compare(idxReader.getLastKey(), end) < 0)) {
+ endOffset = data.length();
+ }
+
+ FileChunk chunk = new FileChunk(data, startOffset, endOffset - startOffset);
+ LOG.info("Retrieve File Chunk: " + chunk);
+ return chunk;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
index 1fe5a0f..67e7423 100644
--- a/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
+++ b/tajo-core/tajo-core-pullserver/src/main/java/org/apache/tajo/pullserver/retriever/AdvancedDataRetriever.java
@@ -22,14 +22,14 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.pullserver.FileAccessForbiddenException;
import org.apache.tajo.util.TajoIdUtils;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
import java.io.File;
import java.io.FileNotFoundException;
@@ -75,13 +75,14 @@ public class AdvancedDataRetriever implements DataRetriever {
if (params.containsKey("sid")) {
List<FileChunk> chunks = Lists.newArrayList();
- List<String> qids = splitMaps(params.get("qid"));
- for (String qid : qids) {
- String[] ids = qid.split("_");
- SubQueryId suid = TajoIdUtils.newSubQueryId(params.get("sid").get(0));
- QueryUnitId quid = new QueryUnitId(suid, Integer.parseInt(ids[0]));
- QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid,
- Integer.parseInt(ids[1]));
+ List<String> queryUnidIds = splitMaps(params.get("qid"));
+ for (String eachQueryUnitId : queryUnidIds) {
+ String[] queryUnitIdSeqTokens = eachQueryUnitId.split("_");
+ ExecutionBlockId ebId = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
+ QueryUnitId quid = new QueryUnitId(ebId, Integer.parseInt(queryUnitIdSeqTokens[0]));
+
+ QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid, Integer.parseInt(queryUnitIdSeqTokens[1]));
+
RetrieverHandler handler = handlerMap.get(attemptId.toString());
FileChunk chunk = handler.get(params);
chunks.add(chunk);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
index 0f6ed41..0896bd1 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/CSVFile.java
@@ -53,7 +53,7 @@ public class CSVFile {
private TableStatistics stats = null;
public CSVAppender(Configuration conf, final TableMeta meta,
- final Path path) throws IOException {
+ final Path path) throws IOException {
super(conf, meta, path);
this.fs = path.getFileSystem(conf);
this.meta = meta;
@@ -94,60 +94,60 @@ public class CSVFile {
} else {
col = schema.getColumn(i);
switch (col.getDataType().getType()) {
- case BOOLEAN:
- sb.append(tuple.getBoolean(i));
- break;
- case BIT:
- sb.append(new String(Base64.encodeBase64(tuple.getByte(i)
- .asByteArray(), false)));
- break;
- case BLOB:
- sb.append(new String(Base64.encodeBase64(tuple.getBytes(i)
- .asByteArray(), false)));
- break;
- case CHAR:
- sb.append(tuple.getChar(i));
- break;
+ case BOOLEAN:
+ sb.append(tuple.getBoolean(i));
+ break;
+ case BIT:
+ sb.append(new String(Base64.encodeBase64(tuple.getByte(i)
+ .asByteArray(), false)));
+ break;
+ case BLOB:
+ sb.append(new String(Base64.encodeBase64(tuple.getBytes(i)
+ .asByteArray(), false)));
+ break;
+ case CHAR:
+ sb.append(tuple.getChar(i));
+ break;
// case STRING:
// sb.append(tuple.getString(i));
// break;
- case TEXT:
- TextDatum td = tuple.getText(i);
- sb.append(td.toString());
- break;
- case INT2:
- sb.append(tuple.getShort(i));
- break;
- case INT4:
- sb.append(tuple.getInt(i));
- break;
- case INT8:
- sb.append(tuple.getLong(i));
- break;
- case FLOAT4:
- sb.append(tuple.getFloat(i));
- break;
- case FLOAT8:
- sb.append(tuple.getDouble(i));
- break;
- case INET4:
- sb.append(tuple.getIPv4(i));
- break;
- case INET6:
- sb.append(tuple.getIPv6(i));
- case ARRAY:
+ case TEXT:
+ TextDatum td = tuple.getText(i);
+ sb.append(td.toString());
+ break;
+ case INT2:
+ sb.append(tuple.getShort(i));
+ break;
+ case INT4:
+ sb.append(tuple.getInt(i));
+ break;
+ case INT8:
+ sb.append(tuple.getLong(i));
+ break;
+ case FLOAT4:
+ sb.append(tuple.getFloat(i));
+ break;
+ case FLOAT8:
+ sb.append(tuple.getDouble(i));
+ break;
+ case INET4:
+ sb.append(tuple.getIPv4(i));
+ break;
+ case INET6:
+ sb.append(tuple.getIPv6(i));
+ case ARRAY:
/*
* sb.append("["); boolean first = true; ArrayDatum array =
* (ArrayDatum) tuple.get(i); for (Datum field : array.toArray()) {
* if (first) { first = false; } else { sb.append(delimiter); }
* sb.append(field.asChars()); } sb.append("]");
*/
- ArrayDatum array = (ArrayDatum) tuple.get(i);
- sb.append(array.toJson());
- break;
- default:
- throw new UnsupportedOperationException("Cannot write such field: "
- + tuple.get(i).type());
+ ArrayDatum array = (ArrayDatum) tuple.get(i);
+ sb.append(array.toJson());
+ break;
+ default:
+ throw new UnsupportedOperationException("Cannot write such field: "
+ + tuple.get(i).type());
}
}
sb.append(delimiter);
@@ -195,7 +195,7 @@ public class CSVFile {
public static class CSVScanner extends FileScanner implements SeekableScanner {
public CSVScanner(Configuration conf, final TableMeta meta,
- final Fragment fragment) throws IOException {
+ final Fragment fragment) throws IOException {
super(conf, meta, fragment);
}
@@ -240,6 +240,11 @@ public class CSVFile {
}
super.init();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("CSVScanner open:" + fragment.getPath() + "," + startOffset + "," + length +
+ "," + fs.getFileStatus(fragment.getPath()).getLen());
+ }
+
if (startOffset != 0) {
fis.seek(startOffset - 1);
while (fis.readByte() != LF) {
@@ -297,7 +302,6 @@ public class CSVFile {
while ((temp[cnt] = fis.readByte()) != LF) {
cnt++;
}
-
// Replace tuple
tuples[tuples.length - 1] = tuples[tuples.length - 1] + new String(temp, 0, cnt);
validIdx = tuples.length;
@@ -319,7 +323,6 @@ public class CSVFile {
this.tupleOffsets[i] = curTupleOffset + this.pageStart;
curTupleOffset += this.tuples[i].getBytes().length + 1;//tuple byte + 1byte line feed
}
-
}
@Override
@@ -351,44 +354,44 @@ public class CSVFile {
tuple.put(tid, DatumFactory.createNullDatum());
} else {
switch (field.getDataType().getType()) {
- case BOOLEAN:
- tuple.put(tid, DatumFactory.createBool(cell));
- break;
- case BIT:
- tuple.put(tid, DatumFactory.createBit(Base64.decodeBase64(cell)[0]));
- break;
- case CHAR:
- tuple.put(tid, DatumFactory.createChar(cell.charAt(0)));
- break;
- case BLOB:
- tuple.put(tid, DatumFactory.createBlob(Base64.decodeBase64(cell)));
- break;
- case INT2:
- tuple.put(tid, DatumFactory.createInt2(cell));
- break;
- case INT4:
- tuple.put(tid, DatumFactory.createInt4(cell));
- break;
- case INT8:
- tuple.put(tid, DatumFactory.createInt8(cell));
- break;
- case FLOAT4:
- tuple.put(tid, DatumFactory.createFloat4(cell));
- break;
- case FLOAT8:
- tuple.put(tid, DatumFactory.createFloat8(cell));
- break;
- case TEXT:
- tuple.put(tid, DatumFactory.createText(cell));
- break;
- case INET4:
- tuple.put(tid, DatumFactory.createInet4(cell));
- break;
- case ARRAY:
- Datum data = StorageGsonHelper.getInstance().fromJson(cell,
- Datum.class);
- tuple.put(tid, data);
- break;
+ case BOOLEAN:
+ tuple.put(tid, DatumFactory.createBool(cell));
+ break;
+ case BIT:
+ tuple.put(tid, DatumFactory.createBit(Base64.decodeBase64(cell)[0]));
+ break;
+ case CHAR:
+ tuple.put(tid, DatumFactory.createChar(cell.charAt(0)));
+ break;
+ case BLOB:
+ tuple.put(tid, DatumFactory.createBlob(Base64.decodeBase64(cell)));
+ break;
+ case INT2:
+ tuple.put(tid, DatumFactory.createInt2(cell));
+ break;
+ case INT4:
+ tuple.put(tid, DatumFactory.createInt4(cell));
+ break;
+ case INT8:
+ tuple.put(tid, DatumFactory.createInt8(cell));
+ break;
+ case FLOAT4:
+ tuple.put(tid, DatumFactory.createFloat4(cell));
+ break;
+ case FLOAT8:
+ tuple.put(tid, DatumFactory.createFloat8(cell));
+ break;
+ case TEXT:
+ tuple.put(tid, DatumFactory.createText(cell));
+ break;
+ case INET4:
+ tuple.put(tid, DatumFactory.createInet4(cell));
+ break;
+ case ARRAY:
+ Datum data = StorageGsonHelper.getInstance().fromJson(cell,
+ Datum.class);
+ tuple.put(tid, data);
+ break;
}
}
}
@@ -430,7 +433,7 @@ public class CSVFile {
int tupleIndex = Arrays.binarySearch(this.tupleOffsets, offset);
if (tupleIndex > -1) {
this.currentIdx = tupleIndex;
- } else if (offset >= this.pageStart + this.bufSize
+ } else if (offset >= this.pageStart + this.bufSize
+ this.prevTailLen - this.tail.length || offset <= this.pageStart) {
fis.seek(offset);
tail = new byte[0];
@@ -441,10 +444,10 @@ public class CSVFile {
// pageBuffer();
} else {
throw new IOException("invalid offset " +
- " < pageStart : " + this.pageStart + " , " +
- " pagelength : " + this.bufSize + " , " +
- " tail lenght : " + this.tail.length +
- " input offset : " + offset + " >");
+ " < pageStart : " + this.pageStart + " , " +
+ " pagelength : " + this.bufSize + " , " +
+ " tail lenght : " + this.tail.length +
+ " input offset : " + offset + " >");
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
index 8b7ec67..a7a1e4a 100644
--- a/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
+++ b/tajo-core/tajo-core-storage/src/main/java/org/apache/tajo/storage/MergeScanner.java
@@ -80,9 +80,13 @@ public class MergeScanner implements Scanner {
@Override
public void close() throws IOException {
- currentScanner.close();
+ if(currentScanner != null) {
+ currentScanner.close();
+ }
iterator = null;
- fragments.clear();
+ if(fragments != null) {
+ fragments.clear();
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/start-tajo.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/start-tajo.sh b/tajo-dist/src/main/bin/start-tajo.sh
index a39ebf1..3d2ad6a 100755
--- a/tajo-dist/src/main/bin/start-tajo.sh
+++ b/tajo-dist/src/main/bin/start-tajo.sh
@@ -26,3 +26,11 @@ bin=`cd "$bin"; pwd`
# start the tajo master daemon
"$bin"/tajo-daemon.sh --config $TAJO_CONF_DIR start master
+
+if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
+ . "${TAJO_CONF_DIR}/tajo-env.sh"
+fi
+
+if [ "$TAJO_WORKER_STANDBY_MODE" = "true" ]; then
+ exec "$bin/tajo-daemons.sh" cd "$TAJO_HOME" \; "$bin/tajo-daemon.sh" start worker
+fi
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/stop-tajo.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/stop-tajo.sh b/tajo-dist/src/main/bin/stop-tajo.sh
index d22ca4a..88587ae 100755
--- a/tajo-dist/src/main/bin/stop-tajo.sh
+++ b/tajo-dist/src/main/bin/stop-tajo.sh
@@ -17,7 +17,7 @@
# limitations under the License.
-# Stop tajo map reduce daemons. Run this on master node.
+# Stop tajo master daemons. Run this on master node.
bin=`dirname "$0"`
bin=`cd "$bin"; pwd`
@@ -25,3 +25,12 @@ bin=`cd "$bin"; pwd`
. "$bin"/tajo-config.sh
"$bin"/tajo-daemon.sh --config $TAJO_CONF_DIR stop master
+
+if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
+ . "${TAJO_CONF_DIR}/tajo-env.sh"
+fi
+
+if [ "$TAJO_WORKER_STANDBY_MODE" = "true" ]; then
+ exec "$bin/tajo-daemons.sh" cd "$TAJO_HOME" \; "$bin/tajo-daemon.sh" stop worker
+fi
+
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo
index a67fe2b..5d5774e 100755
--- a/tajo-dist/src/main/bin/tajo
+++ b/tajo-dist/src/main/bin/tajo
@@ -61,6 +61,7 @@ if [ $# = 0 ]; then
echo "Usage: tajo [--config confdir] COMMAND"
echo "where COMMAND is one of:"
echo " master run the Master Server"
+ echo " worker run the Worker Server"
echo " catalog run the Catalog server"
echo " catutil catalog utility"
echo " cli run the tajo cli"
@@ -272,6 +273,9 @@ if [ "$COMMAND" = "classpath" ] ; then
elif [ "$COMMAND" = "master" ] ; then
CLASS='org.apache.tajo.master.TajoMaster'
TAJO_OPTS="$TAJO_OPTS $TAJO_MASTER_OPTS"
+elif [ "$COMMAND" = "worker" ] ; then
+ CLASS='org.apache.tajo.worker.TajoWorker'
+ TAJO_OPTS="$TAJO_OPTS $TAJO_WORKER_OPTS"
elif [ "$COMMAND" = "catalog" ] ; then
CLASS='org.apache.tajo.catalog.CatalogServer'
TAJO_OPTS="$TAJO_OPTS $TAJO_CATALOG_OPTS"
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo-config.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo-config.sh b/tajo-dist/src/main/bin/tajo-config.sh
index 36c20d8..5972f46 100755
--- a/tajo-dist/src/main/bin/tajo-config.sh
+++ b/tajo-dist/src/main/bin/tajo-config.sh
@@ -53,4 +53,17 @@ then
fi
# Allow alternate conf dir location.
-TAJO_CONF_DIR="${TAJO_CONF_DIR:-$TAJO_HOME/conf}"
\ No newline at end of file
+TAJO_CONF_DIR="${TAJO_CONF_DIR:-$TAJO_HOME/conf}"
+
+#check to see it is specified whether to use the workers or the
+# workers file
+if [ $# -gt 1 ]
+then
+ if [ "--hosts" = "$1" ]
+ then
+ shift
+ workesfile=$1
+ shift
+ export TAJO_WORKERS="${TAJO_CONF_DIR}/$workesfile"
+ fi
+fi
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo-daemon.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo-daemon.sh b/tajo-dist/src/main/bin/tajo-daemon.sh
index 4a86c01..1d4a22b 100755
--- a/tajo-dist/src/main/bin/tajo-daemon.sh
+++ b/tajo-dist/src/main/bin/tajo-daemon.sh
@@ -17,7 +17,7 @@
# limitations under the License.
-# Runs a Hadoop command as a daemon.
+# Runs a Tajo command as a daemon.
#
# Environment Variables
#
@@ -69,11 +69,11 @@ if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
. "${TAJO_CONF_DIR}/tajo-env.sh"
fi
-# Determine if we're starting a secure datanode, and if so, redefine appropriate variables
-if [ "$command" == "datanode" ] && [ "$EUID" -eq 0 ] && [ -n "$TAJO_SECURE_DN_USER" ]; then
- export TAJO_PID_DIR=$TAJO_SECURE_DN_PID_DIR
- export TAJO_LOG_DIR=$TAJO_SECURE_DN_LOG_DIR
- export TAJO_IDENT_STRING=$TAJO_SECURE_DN_USER
+# Determine if we're starting a secure server, and if so, redefine appropriate variables
+if [ "$command" == "worker" ] && [ "$EUID" -eq 0 ] && [ -n "$TAJO_SECURE_WORKER_USER" ]; then
+ export TAJO_PID_DIR=$TAJO_SECURE_WORKER_PID_DIR
+ export TAJO_LOG_DIR=$TAJO_SECURE_WORKER_LOG_DIR
+ export TAJO_IDENT_STRING=$TAJO_SECURE_WORKER_USER
fi
if [ "$TAJO_IDENT_STRING" = "" ]; then
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/bin/tajo-daemons.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo-daemons.sh b/tajo-dist/src/main/bin/tajo-daemons.sh
new file mode 100644
index 0000000..74c599f
--- /dev/null
+++ b/tajo-dist/src/main/bin/tajo-daemons.sh
@@ -0,0 +1,68 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Run a shell command on all worker hosts.
+#
+# Environment Variables
+#
+# TAJO_WORKERS File naming remote hosts.
+# Default is ${TAJO_CONF_DIR}/workers.
+# TAJO_CONF_DIR Alternate conf dir. Default is ${TAJO_HOME}/conf.
+# TAJO_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
+# TAJO_SSH_OPTS Options passed to ssh when running remote commands.
+##
+
+usage="Usage: tajo-daemons.sh command..."
+
+# if no args specified, show usage
+if [ $# -le 0 ]; then
+ echo $usage
+ exit 1
+fi
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/tajo-config.sh
+
+# If the workers file is specified in the command line,
+# then it takes precedence over the definition in
+# neptune-env.sh. Save it here.
+HOSTLIST=$TAJO_WORKERS
+
+if [ -f "${TAJO_CONF_DIR}/tajo-env.sh" ]; then
+ . "${TAJO_CONF_DIR}/tajo-env.sh"
+fi
+
+if [ "$HOSTLIST" = "" ]; then
+ if [ "$TAJO_WORKERS" = "" ]; then
+ export HOSTLIST="${TAJO_CONF_DIR}/workers"
+ else
+ export HOSTLIST="${TAJO_CONF_DIR}"
+ fi
+fi
+
+for slave in `cat "$HOSTLIST"`; do
+ ssh $TAJO_SSH_OPTS $slave $"${@// /\\ }" \
+ 2>&1 | sed "s/^/$slave: /" &
+ if [ "$TAJO_SLAVE_SLEEP" != "" ]; then
+ sleep $TAJO_SLAVE_SLEEP
+ fi
+done
+
+wait
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/conf/tajo-env.sh
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/conf/tajo-env.sh b/tajo-dist/src/main/conf/tajo-env.sh
index 3996030..bca6d2b 100755
--- a/tajo-dist/src/main/conf/tajo-env.sh
+++ b/tajo-dist/src/main/conf/tajo-env.sh
@@ -34,14 +34,23 @@
# export TAJO_OPTS=-server
export TAJO_OPTS=-XX:+PrintGCTimeStamps
+# Extra TajoMaster's java runtime options for TajoMaster. Empty by default
+# export TAJO_MASTER_OPTS=
+
+# Extra TajoWorker's java runtime options for TajoMaster. Empty by default
+# export TAJO_WORKER_OPTS=
+
# Where log files are stored. $TAJO_HOME/logs by default.
# export TAJO_LOG_DIR=${TAJO_HOME}/logs
# The directory where pid files are stored. /tmp by default.
-# export TAJO_PID_DIR=/var/hadoop/pids
+# export TAJO_PID_DIR=/var/tajo/pids
-# A string representing this instance of hadoop. $USER by default.
+# A string representing this instance of tajo. $USER by default.
# export TAJO_IDENT_STRING=$USER
# The scheduling priority for daemon processes. See 'man nice'.
# export TAJO_NICENESS=10
+
+export TAJO_WORKER_STANDBY_MODE=true
+
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-dist/src/main/conf/workers
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/conf/workers b/tajo-dist/src/main/conf/workers
new file mode 100644
index 0000000..d18580b
--- /dev/null
+++ b/tajo-dist/src/main/conf/workers
@@ -0,0 +1 @@
+localhost
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
index b520b3e..154fb9b 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
@@ -143,7 +143,6 @@ public class NettyServerBase {
}
}
if (available(port)) {
- LOG.info("Detect an unused port:" + port);
return port;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
index c58db58..d78e4e1 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcClient.java
@@ -22,10 +22,10 @@ import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.*;
import org.apache.tajo.rpc.RpcProtos.RpcRequest;
import org.apache.tajo.rpc.RpcProtos.RpcResponse;
import org.apache.tajo.util.NetUtils;
+import org.jboss.netty.channel.*;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
@@ -198,8 +198,9 @@ public class ProtoAsyncRpcClient extends NettyClientBase {
@Override
public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
throws Exception {
+ LOG.error(addr + "," + protocol + "," + e.getCause().getMessage(), e.getCause());
e.getChannel().close();
- throw new RemoteException(getErrorMessage(""), e.getCause());
+ throw new RemoteException(getErrorMessage(addr.toString()), e.getCause());
}
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
index 56faaba..1a6ce29 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoAsyncRpcServer.java
@@ -25,9 +25,9 @@ import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.*;
import org.apache.tajo.rpc.RpcProtos.RpcRequest;
import org.apache.tajo.rpc.RpcProtos.RpcResponse;
+import org.jboss.netty.channel.*;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
@@ -118,7 +118,6 @@ public class ProtoAsyncRpcServer extends NettyServerBase {
RemoteCallException callException = (RemoteCallException) e.getCause();
e.getChannel().write(callException.getResponse());
}
-
throw new RemoteException(e.getCause());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
index 2018f6d..2a0b01f 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoBlockingRpcClient.java
@@ -25,10 +25,10 @@ import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.*;
import org.apache.tajo.rpc.RpcProtos.RpcRequest;
import org.apache.tajo.rpc.RpcProtos.RpcResponse;
import org.apache.tajo.util.NetUtils;
+import org.jboss.netty.channel.*;
import java.lang.reflect.Method;
import java.net.InetSocketAddress;
@@ -129,9 +129,13 @@ public class ProtoBlockingRpcClient extends NettyClientBase {
}
private String getErrorMessage(String message) {
- return "Exception [" + protocol.getCanonicalName() +
- "(" + NetUtils.normalizeInetSocketAddress((InetSocketAddress)
- getChannel().getRemoteAddress()) + ")]: " + message;
+ if(protocol != null && getChannel() != null) {
+ return "Exception [" + protocol.getCanonicalName() +
+ "(" + NetUtils.normalizeInetSocketAddress((InetSocketAddress)
+ getChannel().getRemoteAddress()) + ")]: " + message;
+ } else {
+ return "Exception " + message;
+ }
}
private class ClientChannelUpstreamHandler extends SimpleChannelUpstreamHandler {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java b/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
index 6dff443..6a43be3 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/util/NetUtils.java
@@ -99,4 +99,4 @@ public class NetUtils {
}
return host;
}
-}
+}
\ No newline at end of file
[7/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
index 7ae01d6..044f0ae 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ContainerProxy.java
@@ -21,417 +21,52 @@ package org.apache.tajo.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.ContainerManager;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.util.ProtoUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.QueryConf;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.querymaster.QueryMaster;
-import org.apache.tajo.pullserver.PullServerAuxService;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.security.PrivilegedAction;
-import java.util.*;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
public abstract class ContainerProxy {
- private static final Log LOG = LogFactory.getLog(ContainerProxy.class);
+ protected static final Log LOG = LogFactory.getLog(ContainerProxy.class);
final public static FsPermission QUERYCONF_FILE_PERMISSION =
FsPermission.createImmutable((short) 0644); // rw-r--r--
- private final static RecordFactory recordFactory =
- RecordFactoryProvider.getRecordFactory(null);
- private static enum ContainerState {
+ protected static enum ContainerState {
PREP, FAILED, RUNNING, DONE, KILLED_BEFORE_LAUNCH
}
- private final YarnRPC yarnRPC;
- private Configuration conf;
- private QueryMaster.QueryContext context;
+ protected final ExecutionBlockId executionBlockId;
+ protected Configuration conf;
+ protected QueryMasterTask.QueryContext context;
- private ContainerState state;
+ protected ContainerState state;
// store enough information to be able to cleanup the container
- private Container container;
- private ContainerId containerID;
- final private String containerMgrAddress;
- private ContainerToken containerToken;
- private String hostName;
- private int port = -1;
+ protected Container container;
+ protected ContainerId containerID;
+ protected String hostName;
+ protected int port = -1;
- protected abstract void containerStarted();
- protected abstract String getId();
- protected abstract String getRunnerClass();
- protected abstract Vector<CharSequence> getTaskParams();
+ public abstract void launch(ContainerLaunchContext containerLaunchContext);
+ public abstract void stopContainer();
- public ContainerProxy(QueryMaster.QueryContext context, Configuration conf, YarnRPC yarnRPC, Container container) {
+ public ContainerProxy(QueryMasterTask.QueryContext context, Configuration conf,
+ ExecutionBlockId executionBlockId, Container container) {
this.context = context;
this.conf = conf;
- this.yarnRPC = yarnRPC;
this.state = ContainerState.PREP;
this.container = container;
+ this.executionBlockId = executionBlockId;
this.containerID = container.getId();
- NodeId nodeId = container.getNodeId();
- this.containerMgrAddress = nodeId.getHost() + ":" + nodeId.getPort();
- this.containerToken = container.getContainerToken();
- }
-
- protected ContainerManager getCMProxy(ContainerId containerID,
- final String containerManagerBindAddr,
- ContainerToken containerToken)
- throws IOException {
- String [] hosts = containerManagerBindAddr.split(":");
- final InetSocketAddress cmAddr =
- new InetSocketAddress(hosts[0], Integer.parseInt(hosts[1]));
- UserGroupInformation user = UserGroupInformation.getCurrentUser();
-
- if (UserGroupInformation.isSecurityEnabled()) {
- Token<ContainerTokenIdentifier> token =
- ProtoUtils.convertFromProtoFormat(containerToken, cmAddr);
- // the user in createRemoteUser in this context has to be ContainerID
- user = UserGroupInformation.createRemoteUser(containerID.toString());
- user.addToken(token);
- }
-
- ContainerManager proxy = user.doAs(new PrivilegedAction<ContainerManager>() {
- @Override
- public ContainerManager run() {
- return (ContainerManager) yarnRPC.getProxy(ContainerManager.class, cmAddr, conf);
- }
- });
-
- return proxy;
}
public synchronized boolean isCompletelyDone() {
return state == ContainerState.DONE || state == ContainerState.FAILED;
}
- @SuppressWarnings("unchecked")
- public synchronized void launch(ContainerLaunchContext commonContainerLaunchContext) {
- LOG.info("Launching Container with Id: " + containerID);
- if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
- state = ContainerState.DONE;
- LOG.error("Container (" + containerID + " was killed before it was launched");
- return;
- }
-
- ContainerManager proxy = null;
- try {
-
- proxy = getCMProxy(containerID, containerMgrAddress,
- containerToken);
-
- // Construct the actual Container
- ContainerLaunchContext containerLaunchContext = createContainerLaunchContext(commonContainerLaunchContext);
-
- // Now launch the actual container
- StartContainerRequest startRequest = Records
- .newRecord(StartContainerRequest.class);
- startRequest.setContainerLaunchContext(containerLaunchContext);
- StartContainerResponse response = proxy.startContainer(startRequest);
-
- ByteBuffer portInfo = response
- .getServiceResponse(PullServerAuxService.PULLSERVER_SERVICEID);
-
- if(portInfo != null) {
- port = PullServerAuxService.deserializeMetaData(portInfo);
- }
-
- LOG.info("PullServer port returned by ContainerManager for "
- + containerID + " : " + port);
-
- if(port < 0) {
- this.state = ContainerState.FAILED;
- throw new IllegalStateException("Invalid shuffle port number "
- + port + " returned for " + containerID);
- }
-
- containerStarted();
-
- this.state = ContainerState.RUNNING;
- this.hostName = containerMgrAddress.split(":")[0];
- context.addContainer(containerID, this);
- } catch (Throwable t) {
- String message = "Container launch failed for " + containerID + " : "
- + StringUtils.stringifyException(t);
- this.state = ContainerState.FAILED;
- LOG.error(message);
- } finally {
- if (proxy != null) {
- yarnRPC.stopProxy(proxy, conf);
- }
- }
- }
-
- public synchronized void kill() {
-
- if(isCompletelyDone()) {
- return;
- }
- if(this.state == ContainerState.PREP) {
- this.state = ContainerState.KILLED_BEFORE_LAUNCH;
- } else {
- LOG.info("KILLING " + containerID);
-
- ContainerManager proxy = null;
- try {
- proxy = getCMProxy(this.containerID, this.containerMgrAddress,
- this.containerToken);
-
- // kill the remote container if already launched
- StopContainerRequest stopRequest = Records
- .newRecord(StopContainerRequest.class);
- stopRequest.setContainerId(this.containerID);
- proxy.stopContainer(stopRequest);
- // If stopContainer returns without an error, assuming the stop made
- // it over to the NodeManager.
-// context.getEventHandler().handle(
-// new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
- context.removeContainer(containerID);
- } catch (Throwable t) {
-
- // ignore the cleanup failure
- String message = "cleanup failed for container "
- + this.containerID + " : "
- + StringUtils.stringifyException(t);
- LOG.warn(message);
- this.state = ContainerState.DONE;
- return;
- } finally {
- if (proxy != null) {
- yarnRPC.stopProxy(proxy, conf);
- }
- }
- this.state = ContainerState.DONE;
- }
- }
-
- public static ContainerLaunchContext createCommonContainerLaunchContext(Configuration config, String queryId, boolean isMaster) {
- TajoConf conf = (TajoConf)config;
-
- ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
- try {
- ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
- } catch (IOException e) {
- e.printStackTrace();
- }
-
- ////////////////////////////////////////////////////////////////////////////
- // Set the env variables to be setup
- ////////////////////////////////////////////////////////////////////////////
- LOG.info("Set the environment for the application master");
-
- Map<String, String> environment = new HashMap<String, String>();
- //String initialClassPath = getInitialClasspath(conf);
- environment.put(ApplicationConstants.Environment.SHELL.name(), "/bin/bash");
- if(System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()) != null) {
- environment.put(ApplicationConstants.Environment.JAVA_HOME.name(),
- System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()));
- }
-
- // TODO - to be improved with org.apache.tajo.sh shell script
- Properties prop = System.getProperties();
-
- if (prop.getProperty("tajo.test", "FALSE").equalsIgnoreCase("TRUE") ||
- (System.getenv("tajo.test") != null && System.getenv("tajo.test").equalsIgnoreCase("TRUE"))) {
- LOG.info("tajo.test is TRUE");
- environment.put(ApplicationConstants.Environment.CLASSPATH.name(), prop.getProperty("java.class.path", null));
- environment.put("tajo.test", "TRUE");
- } else {
- // Add AppMaster.jar location to classpath
- // At some point we should not be required to add
- // the hadoop specific classpaths to the env.
- // It should be provided out of the box.
- // For now setting all required classpaths including
- // the classpath to "." for the application jar
- StringBuilder classPathEnv = new StringBuilder("./");
- //for (String c : conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
- for (String c : YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH) {
- classPathEnv.append(':');
- classPathEnv.append(c.trim());
- }
-
- classPathEnv.append(":" + System.getenv("TAJO_BASE_CLASSPATH"));
- classPathEnv.append(":./log4j.properties:./*");
- if(System.getenv("HADOOP_HOME") != null) {
- environment.put("HADOOP_HOME", System.getenv("HADOOP_HOME"));
- environment.put(ApplicationConstants.Environment.HADOOP_COMMON_HOME.name(), System.getenv("HADOOP_HOME"));
- environment.put(ApplicationConstants.Environment.HADOOP_HDFS_HOME.name(), System.getenv("HADOOP_HOME"));
- environment.put(ApplicationConstants.Environment.HADOOP_YARN_HOME.name(), System.getenv("HADOOP_HOME"));
- }
-
- if(System.getenv("TAJO_BASE_CLASSPATH") != null) {
- environment.put("TAJO_BASE_CLASSPATH", System.getenv("TAJO_BASE_CLASSPATH"));
- }
- environment.put(ApplicationConstants.Environment.CLASSPATH.name(), classPathEnv.toString());
- }
-
- ctx.setEnvironment(environment);
-
- ////////////////////////////////////////////////////////////////////////////
- // Set the local resources
- ////////////////////////////////////////////////////////////////////////////
- Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
- FileSystem fs = null;
-
- LOG.info("defaultFS: " + conf.get("fs.defaultFS"));
-
- try {
- fs = FileSystem.get(conf);
- } catch (IOException e) {
- LOG.error(e.getMessage(), e);
- }
-
- FileContext fsCtx = null;
- try {
- fsCtx = FileContext.getFileContext(conf);
- } catch (UnsupportedFileSystemException e) {
- LOG.error(e.getMessage(), e);
- }
-
-
- LOG.info("Writing a QueryConf to HDFS and add to local environment");
- try {
- // TODO move to tajo temp
- Path warehousePath = new Path(conf.getVar(TajoConf.ConfVars.ROOT_DIR), TajoConstants.WAREHOUSE_DIR);
- Path queryConfPath = new Path(warehousePath, queryId);
- if(isMaster) {
- queryConfPath = new Path(queryConfPath, QueryConf.QUERY_MASTER_FILENAME);
- } else {
- queryConfPath = new Path(queryConfPath, QueryConf.FILENAME);
- }
-
- if(!fs.exists(queryConfPath)){
- writeConf(conf, queryConfPath);
- } else {
- LOG.warn("QueryConf already exist. path: " + queryConfPath.toString());
- }
- LocalResource queryConfSrc = createApplicationResource(fsCtx, queryConfPath, LocalResourceType.FILE);
-
- localResources.put(queryConfPath.getName(), queryConfSrc);
- ctx.setLocalResources(localResources);
-
- } catch (IOException e) {
- LOG.error(e.getMessage(), e);
- }
-
- // TODO - move to sub-class
- // Add shuffle token
- Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
- try {
- serviceData.put(PullServerAuxService.PULLSERVER_SERVICEID, PullServerAuxService.serializeMetaData(0));
- } catch (IOException ioe) {
- LOG.error(ioe);
- }
- ctx.setServiceData(serviceData);
-
- return ctx;
- }
-
- private static LocalResource createApplicationResource(FileContext fs,
- Path p, LocalResourceType type)
- throws IOException {
- LocalResource rsrc = recordFactory.newRecordInstance(LocalResource.class);
- FileStatus rsrcStat = fs.getFileStatus(p);
- rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs
- .getDefaultFileSystem().resolvePath(rsrcStat.getPath())));
- rsrc.setSize(rsrcStat.getLen());
- rsrc.setTimestamp(rsrcStat.getModificationTime());
- rsrc.setType(type);
- rsrc.setVisibility(LocalResourceVisibility.APPLICATION);
- return rsrc;
- }
-
- private static void writeConf(Configuration conf, Path queryConfFile)
- throws IOException {
- // Write job file to Tajo's fs
- FileSystem fs = queryConfFile.getFileSystem(conf);
- FSDataOutputStream out =
- FileSystem.create(fs, queryConfFile,
- new FsPermission(QUERYCONF_FILE_PERMISSION));
- try {
- conf.writeXml(out);
- } finally {
- out.close();
- }
- }
-
- public ContainerLaunchContext createContainerLaunchContext(ContainerLaunchContext commonContainerLaunchContext) {
- // Setup environment by cloning from common env.
- Map<String, String> env = commonContainerLaunchContext.getEnvironment();
- Map<String, String> myEnv = new HashMap<String, String>(env.size());
- myEnv.putAll(env);
-
- // Duplicate the ByteBuffers for access by multiple containers.
- Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
- for (Map.Entry<String, ByteBuffer> entry : commonContainerLaunchContext.getServiceData().entrySet()) {
- myServiceData.put(entry.getKey(), entry.getValue().duplicate());
- }
-
- ////////////////////////////////////////////////////////////////////////////
- // Set the local resources
- ////////////////////////////////////////////////////////////////////////////
- // Set the necessary command to execute the application master
- Vector<CharSequence> vargs = new Vector<CharSequence>(30);
-
- // Set java executable command
- //LOG.info("Setting up app master command");
- vargs.add("${JAVA_HOME}" + "/bin/java");
- // Set Xmx based on am memory size
- vargs.add("-Xmx2000m");
- // Set Remote Debugging
- //if (!context.getQuery().getSubQuery(event.getSubQueryId()).isLeafQuery()) {
- //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
- //}
- // Set class name
- vargs.add(getRunnerClass());
- vargs.add(getId()); // subqueryId
- vargs.add(containerMgrAddress); // nodeId
- vargs.add(containerID.toString()); // containerId
- Vector<CharSequence> taskParams = getTaskParams();
- if(taskParams != null) {
- vargs.addAll(taskParams);
- }
-
- vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
- vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
-
- // Get final commmand
- StringBuilder command = new StringBuilder();
- for (CharSequence str : vargs) {
- command.append(str).append(" ");
- }
-
- LOG.info("Completed setting up TaskRunner command " + command.toString());
- List<String> commands = new ArrayList<String>();
- commands.add(command.toString());
-
- return BuilderUtils.newContainerLaunchContext(containerID, commonContainerLaunchContext.getUser(),
- container.getResource(), commonContainerLaunchContext.getLocalResources(), myEnv, commands,
- myServiceData, null, new HashMap<ApplicationAccessType, String>());
- }
-
public String getTaskHostName() {
return this.hostName;
}
@@ -439,4 +74,8 @@ public abstract class ContainerProxy {
public int getTaskPort() {
return this.port;
}
+
+ public String getId() {
+ return executionBlockId.toString();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
index 36327ff..a92ef75 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
@@ -15,7 +15,7 @@
package org.apache.tajo.master;
import com.google.common.base.Preconditions;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.engine.planner.logical.*;
@@ -40,7 +40,7 @@ public class ExecutionBlock {
RANGE
}
- private SubQueryId subQueryId;
+ private ExecutionBlockId executionBlockId;
private LogicalNode plan = null;
private StoreTableNode store = null;
private List<ScanNode> scanlist = new ArrayList<ScanNode>();
@@ -50,12 +50,12 @@ public class ExecutionBlock {
private boolean hasJoinPlan;
private boolean hasUnionPlan;
- public ExecutionBlock(SubQueryId subQueryId) {
- this.subQueryId = subQueryId;
+ public ExecutionBlock(ExecutionBlockId executionBlockId) {
+ this.executionBlockId = executionBlockId;
}
- public SubQueryId getId() {
- return subQueryId;
+ public ExecutionBlockId getId() {
+ return executionBlockId;
}
public String getOutputName() {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 8c3617e..ee4b98d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -23,16 +23,11 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.client.YarnClient;
-import org.apache.hadoop.yarn.client.YarnClientImpl;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.QueryConf;
import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoProtos;
import org.apache.tajo.algebra.Expr;
import org.apache.tajo.catalog.CatalogService;
import org.apache.tajo.catalog.CatalogUtil;
@@ -48,18 +43,19 @@ import org.apache.tajo.engine.exception.UnknownWorkerException;
import org.apache.tajo.engine.parser.SQLAnalyzer;
import org.apache.tajo.engine.planner.*;
import org.apache.tajo.engine.planner.global.GlobalOptimizer;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.engine.planner.logical.CreateTableNode;
+import org.apache.tajo.engine.planner.logical.DropTableNode;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.ipc.ClientProtos;
import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.querymaster.QueryMasterManager;
+import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.querymaster.QueryJobManager;
import org.apache.tajo.storage.StorageManager;
import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.TajoIdUtils;
import java.io.IOException;
import java.sql.SQLException;
-import java.util.EnumSet;
-import java.util.Set;
@SuppressWarnings("unchecked")
public class GlobalEngine extends AbstractService {
@@ -76,9 +72,6 @@ public class GlobalEngine extends AbstractService {
private GlobalPlanner globalPlanner;
private GlobalOptimizer globalOptimizer;
- // Yarn
- protected YarnClient yarnClient;
-
public GlobalEngine(final MasterContext context)
throws IOException {
super(GlobalEngine.class.getName());
@@ -89,7 +82,6 @@ public class GlobalEngine extends AbstractService {
public void start() {
try {
- connectYarnClient();
analyzer = new SQLAnalyzer();
planner = new LogicalPlanner(context.getCatalog());
optimizer = new LogicalOptimizer();
@@ -98,138 +90,60 @@ public class GlobalEngine extends AbstractService {
globalOptimizer = new GlobalOptimizer();
} catch (Throwable t) {
- t.printStackTrace();
+ LOG.error(t.getMessage(), t);
}
super.start();
}
public void stop() {
super.stop();
- if (yarnClient != null) {
- yarnClient.stop();
- }
}
- public QueryId executeQuery(String tql)
+ public ClientProtos.GetQueryStatusResponse executeQuery(String sql)
throws InterruptedException, IOException,
NoSuchQueryIdException, IllegalQueryStatusException,
UnknownWorkerException, EmptyClusterException {
- long querySubmittionTime = context.getClock().getTime();
- LOG.info("SQL: " + tql);
+ LOG.info("SQL: " + sql);
// parse the query
- Expr planningContext = analyzer.parse(tql);
+ Expr planningContext = analyzer.parse(sql);
LogicalRootNode plan = (LogicalRootNode) createLogicalPlan(planningContext);
+ ClientProtos.GetQueryStatusResponse.Builder responseBuilder = ClientProtos.GetQueryStatusResponse.newBuilder();
+
if (PlannerUtil.checkIfDDLPlan(plan)) {
updateQuery(plan.getChild());
- return TajoIdUtils.NullQueryId;
+
+ responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+ responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+ responseBuilder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
} else {
- GetNewApplicationResponse newApp = yarnClient.getNewApplication();
- ApplicationId appId = newApp.getApplicationId();
- QueryId queryId = TajoIdUtils.createQueryId(appId, 0);
-
- LOG.info("Get AppId: " + appId + ", QueryId: " + queryId);
- LOG.info("Setting up application submission context for ASM");
-
- //request QueryMaster container
- QueryConf queryConf = new QueryConf(context.getConf());
- queryConf.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
- // the output table is given by user
- if (plan.getChild().getType() == NodeType.CREATE_TABLE) {
- CreateTableNode createTableNode = (CreateTableNode) plan.getChild();
- queryConf.setOutputTable(createTableNode.getTableName());
+ QueryJobManager queryJobManager = context.getQueryJobManager();
+ QueryInfo queryInfo = null;
+ try {
+ queryInfo = queryJobManager.createNewQueryJob(sql, plan);
+ } catch (Exception e) {
+ responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto());
+ responseBuilder.setResultCode(ClientProtos.ResultCode.ERROR);
+ responseBuilder.setState(TajoProtos.QueryState.QUERY_ERROR);
+ responseBuilder.setErrorMessage(StringUtils.stringifyException(e));
+
+ return responseBuilder.build();
}
- QueryMasterManager queryMasterManager = new QueryMasterManager(context, yarnClient, queryId, tql, plan, appId,
- context.getClock(), querySubmittionTime);
- queryMasterManager.init(queryConf);
- queryMasterManager.start();
- context.addQuery(queryId, queryMasterManager);
-
- return queryId;
- }
- }
-
- private ApplicationAttemptId submitQuery() throws YarnRemoteException {
- GetNewApplicationResponse newApp = getNewApplication();
- ApplicationId appId = newApp.getApplicationId();
- LOG.info("Get AppId: " + appId);
- LOG.info("Setting up application submission context for ASM");
- ApplicationSubmissionContext appContext = Records
- .newRecord(ApplicationSubmissionContext.class);
+ //queryJobManager.getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInfo));
- // set the application id
- appContext.setApplicationId(appId);
- // set the application name
- appContext.setApplicationName("Tajo");
-
- org.apache.hadoop.yarn.api.records.Priority
- pri = Records.newRecord(org.apache.hadoop.yarn.api.records.Priority.class);
- pri.setPriority(5);
- appContext.setPriority(pri);
-
- // Set the queue to which this application is to be submitted in the RM
- appContext.setQueue("default");
-
- ContainerLaunchContext amContainer = Records
- .newRecord(ContainerLaunchContext.class);
- appContext.setAMContainerSpec(amContainer);
-
- LOG.info("Submitting application to ASM");
- yarnClient.submitApplication(appContext);
-
- ApplicationReport appReport = monitorApplication(appId,
- EnumSet.of(YarnApplicationState.ACCEPTED));
- ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
- LOG.info("Launching application with id: " + attemptId);
-
- return attemptId;
- }
+ responseBuilder.setQueryId(queryInfo.getQueryId().getProto());
+ responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
+ responseBuilder.setState(queryInfo.getQueryState());
+ if(queryInfo.getQueryMasterHost() != null) {
+ responseBuilder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+ }
+ responseBuilder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
+ }
- private ApplicationAttemptId submitQueryOld() throws YarnRemoteException {
- GetNewApplicationResponse newApp = getNewApplication();
- // Get a new application id
- ApplicationId appId = newApp.getApplicationId();
- LOG.info("Get AppId: " + appId);
- LOG.info("Setting up application submission context for ASM");
- ApplicationSubmissionContext appContext = Records
- .newRecord(ApplicationSubmissionContext.class);
-
- // set the application id
- appContext.setApplicationId(appId);
- // set the application name
- appContext.setApplicationName("Tajo");
-
- // Set the priority for the application master
- org.apache.hadoop.yarn.api.records.Priority
- pri = Records.newRecord(org.apache.hadoop.yarn.api.records.Priority.class);
- pri.setPriority(5);
- appContext.setPriority(pri);
-
- // Set the queue to which this application is to be submitted in the RM
- appContext.setQueue("default");
-
- // Set up the container launch context for the application master
- ContainerLaunchContext amContainer = Records
- .newRecord(ContainerLaunchContext.class);
- appContext.setAMContainerSpec(amContainer);
-
- // unmanaged AM
- appContext.setUnmanagedAM(true);
- LOG.info("Setting unmanaged AM");
-
- // Submit the application to the applications manager
- LOG.info("Submitting application to ASM");
- yarnClient.submitApplication(appContext);
-
- // Monitor the application to wait for launch state
- ApplicationReport appReport = monitorApplication(appId,
- EnumSet.of(YarnApplicationState.ACCEPTED));
- ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
- LOG.info("Launching application with id: " + attemptId);
-
- return attemptId;
+ ClientProtos.GetQueryStatusResponse response = responseBuilder.build();
+ return response;
}
public QueryId updateQuery(String sql) throws IOException, SQLException {
@@ -242,7 +156,7 @@ public class GlobalEngine extends AbstractService {
throw new SQLException("This is not update query:\n" + sql);
} else {
updateQuery(plan.getChild());
- return TajoIdUtils.NullQueryId;
+ return QueryIdFactory.NULL_QUERY_ID;
}
}
@@ -272,24 +186,14 @@ public class GlobalEngine extends AbstractService {
} catch (PlanningException e) {
LOG.error(e.getMessage(), e);
}
- LOG.info("LogicalPlan:\n" + plan.getRootBlock().getRoot());
- return optimizedPlan;
- }
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("LogicalPlan:\n" + plan.getRootBlock().getRoot());
+ }
- private MasterPlan createGlobalPlan(QueryId id, LogicalRootNode rootNode)
- throws IOException {
- MasterPlan globalPlan = globalPlanner.build(id, rootNode);
- return globalOptimizer.optimize(globalPlan);
+ return optimizedPlan;
}
-// private void startQuery(final QueryId queryId, final QueryConf queryConf,
-// final QueryMaster query) {
-// context.getAllQueries().put(queryId, query);
-// query.init(queryConf);
-// query.start();
-// }
-
private TableDesc createTable(CreateTableNode createTable) throws IOException {
TableMeta meta;
@@ -368,58 +272,4 @@ public class GlobalEngine extends AbstractService {
LOG.info("Table \"" + tableName + "\" is dropped.");
}
-
- private void connectYarnClient() {
- this.yarnClient = new YarnClientImpl();
- this.yarnClient.init(context.getConf());
- this.yarnClient.start();
- }
-
- public GetNewApplicationResponse getNewApplication()
- throws YarnRemoteException {
- return yarnClient.getNewApplication();
- }
-
- /**
- * Monitor the submitted application for completion. Kill application if time
- * expires.
- *
- * @param appId
- * Application Id of application to be monitored
- * @return true if application completed successfully
- * @throws YarnRemoteException
- */
- private ApplicationReport monitorApplication(ApplicationId appId,
- Set<YarnApplicationState> finalState) throws YarnRemoteException {
-
- while (true) {
-
- // Check app status every 1 second.
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e) {
- LOG.debug("Thread sleep in monitoring loop interrupted");
- }
-
- // Get application report for the appId we are interested in
- ApplicationReport report = yarnClient.getApplicationReport(appId);
-
- LOG.info("Got application report from ASM for" + ", appId="
- + appId.getId() + ", appAttemptId="
- + report.getCurrentApplicationAttemptId() + ", clientToken="
- + report.getClientToken() + ", appDiagnostics="
- + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
- + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
- + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
- + ", yarnAppState=" + report.getYarnApplicationState().toString()
- + ", distributedFinalState="
- + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
- + report.getTrackingUrl() + ", appUser=" + report.getUser());
-
- YarnApplicationState state = report.getYarnApplicationState();
- if (finalState.contains(state)) {
- return report;
- }
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
index 8b6ba94..14c8bfd 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
@@ -21,9 +21,9 @@ package org.apache.tajo.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryId;
import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
import org.apache.tajo.conf.TajoConf;
@@ -75,9 +75,9 @@ public class GlobalPlanner {
// insert store at the subnode of the root
UnaryNode root = rootNode;
if (root.getChild().getType() != NodeType.STORE) {
- SubQueryId subQueryId = QueryIdFactory.newSubQueryId(this.queryId);
- outputTableName = subQueryId.toString();
- insertStore(subQueryId.toString(),root).setLocal(false);
+ ExecutionBlockId executionBlockId = QueryIdFactory.newExecutionBlockId(this.queryId);
+ outputTableName = executionBlockId.toString();
+ insertStore(executionBlockId.toString(),root).setLocal(false);
}
// convert 2-phase plan
@@ -113,10 +113,10 @@ public class GlobalPlanner {
if (groupby.getChild().getType() != NodeType.UNION &&
groupby.getChild().getType() != NodeType.STORE &&
groupby.getChild().getType() != NodeType.SCAN) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
insertStore(tableId, groupby);
}
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
// insert (a store for the first group by) and (a second group by)
PlannerUtil.transformGroupbyTo2PWithStore((GroupbyNode)node, tableId);
} else if (node.getType() == NodeType.SORT) {
@@ -126,10 +126,10 @@ public class GlobalPlanner {
if (sort.getChild().getType() != NodeType.UNION &&
sort.getChild().getType() != NodeType.STORE &&
sort.getChild().getType() != NodeType.SCAN) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
insertStore(tableId, sort);
}
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
// insert (a store for the first sort) and (a second sort)
PlannerUtil.transformSortTo2PWithStore((SortNode)node, tableId);
} else if (node.getType() == NodeType.JOIN) {
@@ -138,8 +138,8 @@ public class GlobalPlanner {
JoinNode join = (JoinNode) node;
/*
- if (join.getOuterNode().getType() == NodeType.SCAN &&
- join.getInnerNode().getType() == NodeType.SCAN) {
+ if (join.getOuterNode().getType() == ExprType.SCAN &&
+ join.getInnerNode().getType() == ExprType.SCAN) {
ScanNode outerScan = (ScanNode) join.getOuterNode();
ScanNode innerScan = (ScanNode) join.getInnerNode();
@@ -198,14 +198,14 @@ public class GlobalPlanner {
// insert stores for the first phase
if (join.getLeftChild().getType() != NodeType.UNION &&
join.getLeftChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
store = new StoreTableNode(tableId);
store.setLocal(true);
PlannerUtil.insertOuterNode(node, store);
}
if (join.getRightChild().getType() != NodeType.UNION &&
join.getRightChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
store = new StoreTableNode(tableId);
store.setLocal(true);
PlannerUtil.insertInnerNode(node, store);
@@ -216,7 +216,7 @@ public class GlobalPlanner {
// insert stores
if (union.getLeftChild().getType() != NodeType.UNION &&
union.getLeftChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
store = new StoreTableNode(tableId);
if(union.getLeftChild().getType() == NodeType.GROUP_BY) {
/*This case is for cube by operator
@@ -230,7 +230,7 @@ public class GlobalPlanner {
}
if (union.getRightChild().getType() != NodeType.UNION &&
union.getRightChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
store = new StoreTableNode(tableId);
if(union.getRightChild().getType() == NodeType.GROUP_BY) {
/*This case is for cube by operator
@@ -246,7 +246,7 @@ public class GlobalPlanner {
UnaryNode unary = (UnaryNode)node;
if (unary.getType() != NodeType.STORE &&
unary.getChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newSubQueryId(queryId).toString();
+ tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
insertStore(tableId, unary);
}
}
@@ -283,11 +283,11 @@ public class GlobalPlanner {
if (node.getType() == NodeType.STORE) {
store = (StoreTableNode) node;
- SubQueryId id;
- if (store.getTableName().startsWith(QueryId.PREFIX)) {
- id = TajoIdUtils.newSubQueryId(store.getTableName());
+ ExecutionBlockId id;
+ if (store.getTableName().startsWith(ExecutionBlockId.EB_ID_PREFIX)) {
+ id = TajoIdUtils.createExecutionBlockId(store.getTableName());
} else {
- id = QueryIdFactory.newSubQueryId(queryId);
+ id = QueryIdFactory.newExecutionBlockId(queryId);
}
subQuery = new ExecutionBlock(id);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
new file mode 100644
index 0000000..8f83557
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.AbstractService;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TajoAsyncDispatcher extends AbstractService implements Dispatcher {
+
+ private static final Log LOG = LogFactory.getLog(TajoAsyncDispatcher.class);
+
+ private final BlockingQueue<Event> eventQueue;
+ private volatile boolean stopped = false;
+
+ private Thread eventHandlingThread;
+ protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers;
+ private boolean exitOnDispatchException;
+
+ private String id;
+
+ public TajoAsyncDispatcher(String id) {
+ this(id, new LinkedBlockingQueue<Event>());
+ }
+
+ public TajoAsyncDispatcher(String id, BlockingQueue<Event> eventQueue) {
+ super(TajoAsyncDispatcher.class.getName());
+ this.id = id;
+ this.eventQueue = eventQueue;
+ this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>();
+ }
+
+ Runnable createThread() {
+ return new Runnable() {
+ @Override
+ public void run() {
+ while (!stopped && !Thread.currentThread().isInterrupted()) {
+ Event event;
+ try {
+ event = eventQueue.take();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug(id + ",event take:" + event.getType() + "," + event);
+ }
+ } catch(InterruptedException ie) {
+ if (!stopped) {
+ LOG.warn("AsyncDispatcher thread interrupted", ie);
+ }
+ return;
+ }
+ dispatch(event);
+ }
+ }
+ };
+ }
+
+ @Override
+ public synchronized void init(Configuration conf) {
+ this.exitOnDispatchException =
+ conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
+ Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ //start all the components
+ super.start();
+ eventHandlingThread = new Thread(createThread());
+ eventHandlingThread.setName("AsyncDispatcher event handler");
+ eventHandlingThread.start();
+
+ LOG.info("AsyncDispatcher started:" + id);
+ }
+
+ @Override
+ public synchronized void stop() {
+ if(stopped) {
+ return;
+ }
+ stopped = true;
+ if (eventHandlingThread != null) {
+ eventHandlingThread.interrupt();
+ try {
+ eventHandlingThread.join();
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted Exception while stopping", ie);
+ }
+ }
+
+ // stop all the components
+ super.stop();
+
+ LOG.info("AsyncDispatcher stopped:" + id);
+ }
+
+ @SuppressWarnings("unchecked")
+ protected void dispatch(Event event) {
+ //all events go thru this loop
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Dispatching the event " + event.getClass().getName() + "."
+ + event.toString());
+ }
+// LOG.info("====> Dispatching the event " + event.getClass().getName() + "."
+// + event.toString() );
+ Class<? extends Enum> type = event.getType().getDeclaringClass();
+
+ try{
+ EventHandler handler = eventDispatchers.get(type);
+ if(handler != null) {
+ handler.handle(event);
+ } else {
+ throw new Exception("No handler for registered for " + type);
+ }
+ } catch (Throwable t) {
+ //TODO Maybe log the state of the queue
+ LOG.fatal("Error in dispatcher thread:" + event.getType(), t);
+ if (exitOnDispatchException && (ShutdownHookManager.get().isShutdownInProgress()) == false) {
+ LOG.info("Exiting, bye..");
+ System.exit(-1);
+ }
+ } finally {
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void register(Class<? extends Enum> eventType,
+ EventHandler handler) {
+ /* check to see if we have a listener registered */
+ EventHandler<Event> registeredHandler = (EventHandler<Event>)
+ eventDispatchers.get(eventType);
+ LOG.debug("Registering " + eventType + " for " + handler.getClass());
+ if (registeredHandler == null) {
+ eventDispatchers.put(eventType, handler);
+ } else if (!(registeredHandler instanceof MultiListenerHandler)){
+ /* for multiple listeners of an event add the multiple listener handler */
+ MultiListenerHandler multiHandler = new MultiListenerHandler();
+ multiHandler.addHandler(registeredHandler);
+ multiHandler.addHandler(handler);
+ eventDispatchers.put(eventType, multiHandler);
+ } else {
+ /* already a multilistener, just add to it */
+ MultiListenerHandler multiHandler
+ = (MultiListenerHandler) registeredHandler;
+ multiHandler.addHandler(handler);
+ }
+ }
+
+ @Override
+ public EventHandler getEventHandler() {
+ return new GenericEventHandler();
+ }
+
+ class GenericEventHandler implements EventHandler<Event> {
+ public void handle(Event event) {
+ /* all this method does is enqueue all the events onto the queue */
+ int qSize = eventQueue.size();
+ if (qSize !=0 && qSize %1000 == 0) {
+ LOG.info("Size of event-queue is " + qSize);
+ }
+ int remCapacity = eventQueue.remainingCapacity();
+ if (remCapacity < 1000) {
+ LOG.warn("Very low remaining capacity in the event-queue: "
+ + remCapacity);
+ }
+ try {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug(id + ",add event:" +
+ event.getType() + "," + event + "," +
+ (eventHandlingThread == null ? "null" : eventHandlingThread.isAlive()));
+ }
+ eventQueue.put(event);
+ } catch (InterruptedException e) {
+ if (!stopped) {
+ LOG.warn("AsyncDispatcher thread interrupted", e);
+ }
+ throw new YarnException(e);
+ }
+ }
+ }
+
+ /**
+ * Multiplexing an event. Sending it to different handlers that
+ * are interested in the event.
+ */
+ static class MultiListenerHandler implements EventHandler<Event> {
+ List<EventHandler<Event>> listofHandlers;
+
+ public MultiListenerHandler() {
+ listofHandlers = new ArrayList<EventHandler<Event>>();
+ }
+
+ @Override
+ public void handle(Event event) {
+ for (EventHandler<Event> handler: listofHandlers) {
+ handler.handle(event);
+ }
+ }
+
+ void addHandler(EventHandler<Event> handler) {
+ listofHandlers.add(handler);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
new file mode 100644
index 0000000..5359311
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.event.QueryEvent;
+import org.apache.tajo.master.event.QueryEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.rm.TajoWorkerContainer;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.ProtoAsyncRpcClient;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TajoContainerProxy extends ContainerProxy {
+ public TajoContainerProxy(QueryMasterTask.QueryContext context,
+ Configuration conf, Container container,
+ ExecutionBlockId executionBlockId) {
+ super(context, conf, executionBlockId, container);
+ }
+
+ @Override
+ public void launch(ContainerLaunchContext containerLaunchContext) {
+ context.getResourceAllocator().addContainer(containerID, this);
+ this.hostName = container.getNodeId().getHost();
+ this.port = context.getQueryMasterContext().getWorkerContext().getPullService().getPort();
+ this.state = ContainerState.RUNNING;
+
+ assignExecutionBlock(executionBlockId, container);
+
+ context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+ }
+
+ private void assignExecutionBlock(ExecutionBlockId executionBlockId, Container container) {
+ ProtoAsyncRpcClient tajoWorkerRpc = null;
+ try {
+ InetSocketAddress myAddr= context.getQueryMasterContext().getWorkerContext()
+ .getTajoWorkerManagerService().getBindAddr();
+
+ InetSocketAddress addr = new InetSocketAddress(container.getNodeId().getHost(), container.getNodeId().getPort());
+ tajoWorkerRpc = new ProtoAsyncRpcClient(TajoWorkerProtocol.class, addr);
+ TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerRpcClient = tajoWorkerRpc.getStub();
+
+ TajoWorkerProtocol.RunExecutionBlockRequestProto request =
+ TajoWorkerProtocol.RunExecutionBlockRequestProto.newBuilder()
+ .setExecutionBlockId(executionBlockId.toString())
+ .setQueryMasterHost(myAddr.getHostName())
+ .setQueryMasterPort(myAddr.getPort())
+ .setNodeId(container.getNodeId().toString())
+ .setContainerId(container.getId().toString())
+ .setQueryOutputPath(context.getOutputPath().toString())
+ .build();
+
+ tajoWorkerRpcClient.executeExecutionBlock(null, request, NullCallback.get());
+ } catch (Exception e) {
+ //TODO retry
+ LOG.error(e.getMessage(), e);
+ } finally {
+ if(tajoWorkerRpc != null) {
+ (new AyncRpcClose(tajoWorkerRpc)).start();
+ }
+ }
+ }
+
+ class AyncRpcClose extends Thread {
+ ProtoAsyncRpcClient client;
+ public AyncRpcClose(ProtoAsyncRpcClient client) {
+ this.client = client;
+ }
+
+ @Override
+ public void run() {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ }
+ client.close();
+ }
+ }
+
+ @Override
+ public synchronized void stopContainer() {
+ LOG.info("Release TajoWorker Resource: " + executionBlockId + "," + containerID + ", state:" + this.state);
+ if(isCompletelyDone()) {
+ LOG.info("====> Container already stopped:" + containerID);
+ return;
+ }
+ if(this.state == ContainerState.PREP) {
+ this.state = ContainerState.KILLED_BEFORE_LAUNCH;
+ } else {
+ try {
+ releaseWorkerResource(context, executionBlockId, ((TajoWorkerContainer)container).getWorkerResource());
+ context.getResourceAllocator().removeContainer(containerID);
+ this.state = ContainerState.DONE;
+ } catch (Throwable t) {
+ // ignore the cleanup failure
+ String message = "cleanup failed for container "
+ + this.containerID + " : "
+ + StringUtils.stringifyException(t);
+ LOG.warn(message);
+ this.state = ContainerState.DONE;
+ return;
+ }
+ }
+ }
+
+ public static void releaseWorkerResource(QueryMasterTask.QueryContext context,
+ ExecutionBlockId executionBlockId,
+ WorkerResource workerResource) throws Exception {
+ List<WorkerResource> workerResources = new ArrayList<WorkerResource>();
+ workerResources.add(workerResource);
+
+ releaseWorkerResource(context, executionBlockId, workerResources);
+ }
+
+ public static void releaseWorkerResource(QueryMasterTask.QueryContext context,
+ ExecutionBlockId executionBlockId,
+ List<WorkerResource> workerResources) throws Exception {
+ List<TajoMasterProtocol.WorkerResourceProto> workerResourceProtos =
+ new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
+
+ for(WorkerResource eahWorkerResource: workerResources) {
+ workerResourceProtos.add(TajoMasterProtocol.WorkerResourceProto.newBuilder()
+ .setWorkerHostAndPort(eahWorkerResource.getId())
+ .setExecutionBlockId(executionBlockId.getProto())
+ .setMemoryMBSlots(eahWorkerResource.getMemoryMBSlots())
+ .setDiskSlots(eahWorkerResource.getDiskSlots())
+ .build()
+ );
+ }
+ context.getQueryMasterContext().getWorkerContext().getTajoMasterRpcClient()
+ .releaseWorkerResource(null,
+ TajoMasterProtocol.WorkerResourceReleaseRequest.newBuilder()
+ .addAllWorkerResources(workerResourceProtos)
+ .build(),
+ NullCallback.get());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
index b84b51b..f22472a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -18,7 +18,6 @@
package org.apache.tajo.master;
-import com.google.common.collect.Maps;
import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -32,12 +31,9 @@ import org.apache.hadoop.yarn.SystemClock;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.util.RackResolver;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.TajoConstants;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
@@ -47,16 +43,15 @@ import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.engine.function.Country;
import org.apache.tajo.engine.function.InCountry;
import org.apache.tajo.engine.function.builtin.*;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.master.querymaster.QueryMasterManager;
-import org.apache.tajo.master.querymaster.QueryMasterManagerService;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.master.rm.WorkerResourceManager;
import org.apache.tajo.storage.StorageManager;
import org.apache.tajo.webapp.StaticHttpServer;
+import java.lang.reflect.Constructor;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.List;
-import java.util.Map;
public class TajoMaster extends CompositeService {
@@ -79,12 +74,14 @@ public class TajoMaster extends CompositeService {
private GlobalEngine globalEngine;
private AsyncDispatcher dispatcher;
private TajoMasterClientService tajoMasterClientService;
- private QueryMasterManagerService queryMasterManagerService;
- private YarnRPC yarnRPC;
+ private TajoMasterService tajoMasterService;
+ private WorkerResourceManager resourceManager;
//Web Server
private StaticHttpServer webServer;
+ private QueryJobManager queryJobManager;
+
public TajoMaster() throws Exception {
super(TajoMaster.class.getName());
}
@@ -96,16 +93,23 @@ public class TajoMaster extends CompositeService {
context = new MasterContext(conf);
clock = new SystemClock();
-
try {
RackResolver.init(conf);
+// this.conf.writeXml(System.out);
+ String className = this.conf.get("tajo.resource.manager", TajoWorkerResourceManager.class.getCanonicalName());
+ Class<WorkerResourceManager> resourceManagerClass =
+ (Class<WorkerResourceManager>)Class.forName(className);
+
+ Constructor<WorkerResourceManager> constructor = resourceManagerClass.getConstructor(MasterContext.class);
+ resourceManager = constructor.newInstance(context);
+ resourceManager.init(context.getConf());
+
+ //TODO WebServer port configurable
webServer = StaticHttpServer.getInstance(this ,"admin", null, 8080 ,
true, null, context.getConf(), null);
webServer.start();
- QueryIdFactory.reset();
-
// Get the tajo base dir
this.basePath = new Path(conf.getVar(ConfVars.ROOT_DIR));
LOG.info("Tajo Root dir is set " + basePath);
@@ -128,8 +132,6 @@ public class TajoMaster extends CompositeService {
LOG.info("Warehouse dir (" + wareHousePath + ") is created");
}
- yarnRPC = YarnRPC.create(conf);
-
this.dispatcher = new AsyncDispatcher();
addIfService(dispatcher);
@@ -149,15 +151,19 @@ public class TajoMaster extends CompositeService {
globalEngine = new GlobalEngine(context);
addIfService(globalEngine);
+ queryJobManager = new QueryJobManager(context);
+ addIfService(queryJobManager);
+
tajoMasterClientService = new TajoMasterClientService(context);
addIfService(tajoMasterClientService);
- queryMasterManagerService = new QueryMasterManagerService(context);
- addIfService(queryMasterManagerService);
+ tajoMasterService = new TajoMasterService(context);
+ addIfService(tajoMasterService);
} catch (Exception e) {
- e.printStackTrace();
+ LOG.error(e.getMessage(), e);
}
+ LOG.info("====> Tajo master started");
super.init(conf);
}
@@ -282,10 +288,6 @@ public class TajoMaster extends CompositeService {
LOG.error(e);
}
- for(QueryMasterManager eachQuery: getContext().getAllQueries().values()) {
- eachQuery.stop();
- }
-
super.stop();
LOG.info("TajoMaster main thread exiting");
}
@@ -310,40 +312,7 @@ public class TajoMaster extends CompositeService {
return this.storeManager;
}
- // TODO - to be improved
- public Collection<QueryMasterProtocol.TaskStatusProto> getProgressQueries() {
- return null;
- }
-
-// private class QueryEventDispatcher implements EventHandler<QueryEvent> {
-// @Override
-// public void handle(QueryEvent queryEvent) {
-// LOG.info("QueryEvent: " + queryEvent.getQueryId());
-// LOG.info("Found: " + context.getQuery(queryEvent.getQueryId()).getContext().getQueryId());
-// context.getQuery(queryEvent.getQueryId()).handle(queryEvent);
-// }
-// }
-
- public static void main(String[] args) throws Exception {
- StringUtils.startupShutdownMessage(TajoMaster.class, args, LOG);
-
- try {
- TajoMaster master = new TajoMaster();
- ShutdownHookManager.get().addShutdownHook(
- new CompositeServiceShutdownHook(master),
- SHUTDOWN_HOOK_PRIORITY);
- TajoConf conf = new TajoConf(new YarnConfiguration());
- master.init(conf);
- master.start();
- } catch (Throwable t) {
- LOG.fatal("Error starting TajoMaster", t);
- System.exit(-1);
- }
- }
-
public class MasterContext {
- //private final Map<QueryId, QueryMaster> queries = Maps.newConcurrentMap();
- private final Map<QueryId, QueryMasterManager> queries = Maps.newConcurrentMap();
private final TajoConf conf;
public MasterContext(TajoConf conf) {
@@ -358,20 +327,12 @@ public class TajoMaster extends CompositeService {
return clock;
}
- public QueryMasterManager getQuery(QueryId queryId) {
- return queries.get(queryId);
+ public QueryJobManager getQueryJobManager() {
+ return queryJobManager;
}
- public Map<QueryId, QueryMasterManager> getAllQueries() {
- return queries;
- }
-
- public void addQuery(QueryId queryId, QueryMasterManager queryMasterManager) {
- queries.put(queryId, queryMasterManager);
- }
-
- public AsyncDispatcher getDispatcher() {
- return dispatcher;
+ public WorkerResourceManager getResourceManager() {
+ return resourceManager;
}
public EventHandler getEventHandler() {
@@ -390,16 +351,23 @@ public class TajoMaster extends CompositeService {
return storeManager;
}
- public YarnRPC getYarnRPC() {
- return yarnRPC;
+ public TajoMasterService getTajoMasterService() {
+ return tajoMasterService;
}
+ }
- public TajoMasterClientService getClientService() {
- return tajoMasterClientService;
- }
+ public static void main(String[] args) throws Exception {
+ StringUtils.startupShutdownMessage(TajoMaster.class, args, LOG);
- public QueryMasterManagerService getQueryMasterManagerService() {
- return queryMasterManagerService;
+ try {
+ TajoMaster master = new TajoMaster();
+ ShutdownHookManager.get().addShutdownHook(new CompositeServiceShutdownHook(master), SHUTDOWN_HOOK_PRIORITY);
+ TajoConf conf = new TajoConf(new YarnConfiguration());
+ master.init(conf);
+ master.start();
+ } catch (Throwable t) {
+ LOG.fatal("Error starting TajoMaster", t);
+ System.exit(-1);
}
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index ed1376c..8578b64 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -26,9 +26,11 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoIdProtos;
import org.apache.tajo.TajoProtos;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.exception.AlreadyExistsTableException;
@@ -37,18 +39,18 @@ import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto;
import org.apache.tajo.catalog.statistics.TableStat;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.query.exception.SQLSyntaxError;
+import org.apache.tajo.ipc.ClientProtos;
import org.apache.tajo.ipc.ClientProtos.*;
import org.apache.tajo.ipc.TajoMasterClientProtocol;
import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService;
import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.querymaster.QueryMasterManager;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.querymaster.QueryJobManager;
import org.apache.tajo.rpc.ProtoBlockingRpcServer;
import org.apache.tajo.rpc.RemoteException;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
import java.io.IOException;
import java.net.InetSocketAddress;
@@ -88,9 +90,10 @@ public class TajoMasterClientService extends AbstractService {
LOG.error(e);
}
server.start();
- bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
- this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress));
- LOG.info("TajoMasterClientService startup");
+ bindAddress = server.getListenAddress();
+ this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS,
+ org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
+ LOG.info("Instantiated TajoMasterClientService at " + this.bindAddress);
super.start();
}
@@ -99,7 +102,6 @@ public class TajoMasterClientService extends AbstractService {
if (server != null) {
server.shutdown();
}
- LOG.info("TajoMasterClientService shutdown");
super.stop();
}
@@ -123,40 +125,26 @@ public class TajoMasterClientService extends AbstractService {
}
@Override
- public SubmitQueryResponse submitQuery(RpcController controller,
+ public GetQueryStatusResponse submitQuery(RpcController controller,
QueryRequest request)
throws ServiceException {
- QueryId queryId;
- SubmitQueryResponse.Builder build = SubmitQueryResponse.newBuilder();
try {
- queryId = context.getGlobalEngine().executeQuery(request.getQuery());
- } catch (SQLSyntaxError e) {
- build.setResultCode(ResultCode.ERROR);
- build.setErrorMessage(e.getMessage());
- return build.build();
-
- } catch (Exception e) {
- build.setResultCode(ResultCode.ERROR);
- String msg = e.getMessage();
- if (msg == null) {
- msg = "Internal Error";
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Query [" + request.getQuery() + "] is submitted");
}
-
- if (LOG.isDebugEnabled()) {
- LOG.error(msg, e);
+ return context.getGlobalEngine().executeQuery(request.getQuery());
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ ClientProtos.GetQueryStatusResponse.Builder responseBuilder = ClientProtos.GetQueryStatusResponse.newBuilder();
+ responseBuilder.setResultCode(ResultCode.ERROR);
+ if (e.getMessage() != null) {
+ responseBuilder.setErrorMessage(ExceptionUtils.getStackTrace(e));
} else {
- LOG.error(msg);
+ responseBuilder.setErrorMessage("Internal Error");
}
- build.setErrorMessage(msg);
- return build.build();
+ return responseBuilder.build();
}
-
- LOG.info("Query " + queryId + " is submitted");
- build.setResultCode(ResultCode.OK);
- build.setQueryId(queryId.getProto());
-
- return build.build();
}
@Override
@@ -183,13 +171,17 @@ public class TajoMasterClientService extends AbstractService {
GetQueryResultRequest request)
throws ServiceException {
QueryId queryId = new QueryId(request.getQueryId());
- QueryMasterManager queryMasterManager = context.getQuery(queryId);
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+ }
+ QueryInProgress queryInProgress = context.getQueryJobManager().getQueryInProgress(queryId);
+ QueryInfo queryInfo = queryInProgress.getQueryInfo();
GetQueryResultResponse.Builder builder
= GetQueryResultResponse.newBuilder();
- switch (queryMasterManager.getState()) {
+ switch (queryInfo.getQueryState()) {
case QUERY_SUCCEEDED:
- builder.setTableDesc((TableDescProto) queryMasterManager.getResultDesc().getProto());
+ // TODO check this logic needed
+ //builder.setTableDesc((TableDescProto) queryJobManager.getResultDesc().getProto());
break;
case QUERY_FAILED:
case QUERY_ERROR:
@@ -218,23 +210,25 @@ public class TajoMasterClientService extends AbstractService {
QueryId queryId = new QueryId(request.getQueryId());
builder.setQueryId(request.getQueryId());
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
builder.setResultCode(ResultCode.OK);
builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
} else {
- QueryMasterManager queryMasterManager = context.getQuery(queryId);
- if (queryMasterManager != null) {
+ QueryInProgress queryInProgress = context.getQueryJobManager().getQueryInProgress(queryId);
+ if (queryInProgress != null) {
+ QueryInfo queryInfo = queryInProgress.getQueryInfo();
builder.setResultCode(ResultCode.OK);
- builder.setState(queryMasterManager.getState());
- builder.setProgress(queryMasterManager.getProgress());
- builder.setSubmitTime(queryMasterManager.getAppSubmitTime());
- if(queryMasterManager.getQueryMasterHost() != null) {
- builder.setQueryMasterHost(queryMasterManager.getQueryMasterHost());
- builder.setQueryMasterPort(queryMasterManager.getQueryMasterClientPort());
+ builder.setState(queryInfo.getQueryState());
+ builder.setProgress(queryInfo.getProgress());
+ builder.setSubmitTime(queryInfo.getStartTime());
+ if(queryInfo.getQueryMasterHost() != null) {
+ builder.setQueryMasterHost(queryInfo.getQueryMasterHost());
+ builder.setQueryMasterPort(queryInfo.getQueryMasterClientPort());
}
-
- if (queryMasterManager.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
- builder.setFinishTime(queryMasterManager.getFinishTime());
+ //builder.setInitTime(queryJobManager.getInitializationTime());
+ //builder.setHasResult(!queryJobManager.isCreateTableStmt());
+ if (queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+ builder.setFinishTime(queryInfo.getFinishTime());
} else {
builder.setFinishTime(System.currentTimeMillis());
}
@@ -249,11 +243,12 @@ public class TajoMasterClientService extends AbstractService {
@Override
public BoolProto killQuery(RpcController controller,
- ApplicationAttemptIdProto request)
+ TajoIdProtos.QueryIdProto request)
throws ServiceException {
QueryId queryId = new QueryId(request);
- QueryMasterManager queryMasterManager = context.getQuery(queryId);
- //queryMasterManager.handle(new QueryEvent(queryId, QueryEventType.KILL));
+ QueryJobManager queryJobManager = context.getQueryJobManager();
+ //TODO KHJ, change QueryJobManager to event handler
+ //queryJobManager.handle(new QueryEvent(queryId, QueryEventType.KILL));
return BOOL_TRUE;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
new file mode 100644
index 0000000..f0a4618
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoMasterService.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.ProtoAsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+
+public class TajoMasterService extends AbstractService {
+ private final static Log LOG = LogFactory.getLog(TajoMasterService.class);
+
+ private final TajoMaster.MasterContext context;
+ private final TajoConf conf;
+ private final TajoMasterServiceHandler masterHandler;
+ private ProtoAsyncRpcServer server;
+ private InetSocketAddress bindAddress;
+
+ private final BoolProto BOOL_TRUE = BoolProto.newBuilder().setValue(true).build();
+ private final BoolProto BOOL_FALSE = BoolProto.newBuilder().setValue(false).build();
+
+ public TajoMasterService(TajoMaster.MasterContext context) {
+ super(TajoMasterService.class.getName());
+ this.context = context;
+ this.conf = context.getConf();
+ this.masterHandler = new TajoMasterServiceHandler();
+ }
+
+ @Override
+ public void start() {
+ // TODO resolve hostname
+ String confMasterServiceAddr = conf.getVar(TajoConf.ConfVars.TAJO_MASTER_SERVICE_ADDRESS);
+ InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr);
+ try {
+ server = new ProtoAsyncRpcServer(TajoMasterProtocol.class, masterHandler, initIsa);
+ } catch (Exception e) {
+ LOG.error(e);
+ }
+ server.start();
+ bindAddress = server.getListenAddress();
+ this.conf.setVar(TajoConf.ConfVars.TAJO_MASTER_SERVICE_ADDRESS,
+ org.apache.tajo.util.NetUtils.getIpPortString(bindAddress));
+ LOG.info("Instantiated TajoMasterService at " + this.bindAddress);
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(server != null) {
+ server.shutdown();
+ server = null;
+ }
+ super.stop();
+ }
+
+ public InetSocketAddress getBindAddress() {
+ return bindAddress;
+ }
+
+ public class TajoMasterServiceHandler
+ implements TajoMasterProtocol.TajoMasterProtocolService.Interface {
+ @Override
+ public void heartbeat(
+ RpcController controller,
+ TajoMasterProtocol.TajoHeartbeat request, RpcCallback<TajoMasterProtocol.TajoHeartbeatResponse> done) {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Received QueryHeartbeat:" + request.getTajoWorkerHost() + ":" + request.getTajoWorkerPort());
+ }
+
+ TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand command = null;
+ if(request.hasQueryId()) {
+ QueryId queryId = new QueryId(request.getQueryId());
+
+ //heartbeat from querymaster
+ //LOG.info("Received QueryHeartbeat:" + queryId + "," + request);
+ QueryJobManager queryJobManager = context.getQueryJobManager();
+ command = queryJobManager.queryHeartbeat(request);
+ } else {
+ //heartbeat from TajoWorker
+ context.getResourceManager().workerHeartbeat(request);
+ }
+
+ //ApplicationAttemptId attemptId = queryJobManager.getAppAttemptId();
+ //String attemptIdStr = attemptId == null ? null : attemptId.toString();
+ TajoMasterProtocol.TajoHeartbeatResponse.Builder builder = TajoMasterProtocol.TajoHeartbeatResponse.newBuilder();
+ builder.setHeartbeatResult(BOOL_TRUE);
+ if(command != null) {
+ builder.setResponseCommand(command);
+ }
+ done.run(builder.build());
+ }
+
+ @Override
+ public void allocateWorkerResources(
+ RpcController controller,
+ TajoMasterProtocol.WorkerResourceAllocationRequest request,
+ RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> done) {
+ context.getResourceManager().allocateWorkerResources(request, done);
+
+// List<String> workerHosts = new ArrayList<String>();
+// for(WorkerResource eachWorker: workerResources) {
+// workerHosts.add(eachWorker.getAllocatedHost() + ":" + eachWorker.getPorts()[0]);
+// }
+//
+// done.run(TajoMasterProtocol.WorkerResourceAllocationResponse.newBuilder()
+// .setExecutionBlockId(request.getExecutionBlockId())
+// .addAllAllocatedWorks(workerHosts)
+// .build()
+// );
+ }
+
+ @Override
+ public void releaseWorkerResource(RpcController controller,
+ TajoMasterProtocol.WorkerResourceReleaseRequest request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ List<TajoMasterProtocol.WorkerResourceProto> workerResources = request.getWorkerResourcesList();
+ for(TajoMasterProtocol.WorkerResourceProto eachWorkerResource: workerResources) {
+ WorkerResource workerResource = new WorkerResource();
+ String[] tokens = eachWorkerResource.getWorkerHostAndPort().split(":");
+ workerResource.setAllocatedHost(tokens[0]);
+ workerResource.setPorts(new int[]{Integer.parseInt(tokens[1])});
+ workerResource.setMemoryMBSlots(eachWorkerResource.getMemoryMBSlots());
+ workerResource.setDiskSlots(eachWorkerResource.getDiskSlots());
+
+ LOG.info("====> releaseWorkerResource:" + workerResource);
+ context.getResourceManager().releaseWorkerResource(
+ new QueryId(eachWorkerResource.getExecutionBlockId().getQueryId()),
+ workerResource);
+ }
+ done.run(BOOL_TRUE);
+ }
+
+ @Override
+ public void stopQueryMaster(RpcController controller, TajoIdProtos.QueryIdProto request,
+ RpcCallback<BoolProto> done) {
+ context.getQueryJobManager().stopQuery(new QueryId(request));
+ done.run(BOOL_TRUE);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
index 93aaa5d..1e6655c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerGroupEvent.java
@@ -20,7 +20,7 @@ package org.apache.tajo.master;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
import java.util.Collection;
@@ -31,17 +31,21 @@ public class TaskRunnerGroupEvent extends AbstractEvent<EventType> {
CONTAINER_REMOTE_CLEANUP
}
- protected final SubQueryId subQueryId;
+ protected final ExecutionBlockId executionBlockId;
protected final Collection<Container> containers;
public TaskRunnerGroupEvent(EventType eventType,
- SubQueryId subQueryId,
+ ExecutionBlockId executionBlockId,
Collection<Container> containers) {
super(eventType);
- this.subQueryId = subQueryId;
+ this.executionBlockId = executionBlockId;
this.containers = containers;
}
public Collection<Container> getContainers() {
return containers;
}
+
+ public ExecutionBlockId getExecutionBlockId() {
+ return executionBlockId;
+ }
}
[2/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
index a41b280..47ec7bc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunner.java
@@ -26,20 +26,20 @@ import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryConf;
+import org.apache.tajo.QueryId;
import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.TajoProtos.TaskAttemptState;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.engine.query.QueryUnitRequestImpl;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService.Interface;
import org.apache.tajo.rpc.CallFuture2;
import org.apache.tajo.rpc.NullCallback;
import org.apache.tajo.rpc.ProtoAsyncRpcClient;
@@ -51,7 +51,7 @@ import java.security.PrivilegedExceptionAction;
import java.util.Map;
import java.util.concurrent.*;
-import static org.apache.tajo.ipc.QueryMasterProtocol.*;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
/**
* The driver class for Tajo QueryUnit processing.
@@ -60,17 +60,17 @@ public class TaskRunner extends AbstractService {
/** class logger */
private static final Log LOG = LogFactory.getLog(TaskRunner.class);
- private QueryConf conf;
+ private QueryConf queryConf;
private volatile boolean stopped = false;
- private final SubQueryId subQueryId;
- private ApplicationId appId;
- private final NodeId nodeId;
- private final ContainerId containerId;
+ private ExecutionBlockId executionBlockId;
+ private QueryId queryId;
+ private NodeId nodeId;
+ private ContainerId containerId;
// Cluster Management
- private QueryMasterProtocol.QueryMasterProtocolService.Interface master;
+ private TajoWorkerProtocol.TajoWorkerProtocolService.Interface master;
// for temporal or intermediate files
private FileSystem localFS;
@@ -94,7 +94,7 @@ public class TaskRunner extends AbstractService {
private Thread taskLauncher;
// Contains the object references related for TaskRunner
- private WorkerContext workerContext;
+ private TaskRunnerContext taskRunnerContext;
// for the doAs block
private UserGroupInformation taskOwner;
@@ -102,34 +102,89 @@ public class TaskRunner extends AbstractService {
private String baseDir;
private Path baseDirPath;
+ private ProtoAsyncRpcClient client;
+
+ private TaskRunnerManager taskRunnerManager;
+
public TaskRunner(
- final SubQueryId subQueryId,
+ final ExecutionBlockId executionBlockId,
final NodeId nodeId,
UserGroupInformation taskOwner,
Interface master, ContainerId containerId) {
super(TaskRunner.class.getName());
- this.subQueryId = subQueryId;
- this.appId = subQueryId.getQueryId().getApplicationId();
+ this.executionBlockId = executionBlockId;
+ this.queryId = executionBlockId.getQueryId();
this.nodeId = nodeId;
this.taskOwner = taskOwner;
this.master = master;
this.containerId = containerId;
}
- @Override
- public void init(Configuration _conf) {
- this.conf = (QueryConf) _conf;
+ public TaskRunner(TaskRunnerManager taskRunnerManager, QueryConf conf, String[] args) {
+ super(TaskRunner.class.getName());
+ this.taskRunnerManager = taskRunnerManager;
try {
- this.workerContext = new WorkerContext();
+ final ExecutionBlockId executionBlockId = TajoIdUtils.createExecutionBlockId(args[1]);
+
+ conf.setOutputPath(new Path(args[6]));
+
+ LOG.info("NM Local Dir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname));
+ LOG.info("OUTPUT DIR: " + conf.getOutputPath());
+ LOG.info("Tajo Root Dir: " + conf.getVar(ConfVars.ROOT_DIR));
+
+ UserGroupInformation.setConfiguration(conf);
+
+ // QueryBlockId from String
+ // NodeId has a form of hostname:port.
+ NodeId nodeId = ConverterUtils.toNodeId(args[2]);
+ this.containerId = ConverterUtils.toContainerId(args[3]);
+
+ // QueryMaster's address
+ String host = args[4];
+ int port = Integer.parseInt(args[5]);
+ final InetSocketAddress masterAddr = NetUtils.createSocketAddrForHost(host, port);
+
+ LOG.info("QueryMaster Address:" + masterAddr);
+ // TODO - 'load credential' should be implemented
+ // Getting taskOwner
+ UserGroupInformation taskOwner =
+ UserGroupInformation.createRemoteUser(conf.getVar(ConfVars.QUERY_USERNAME));
+ //taskOwner.addToken(token);
+
+ // initialize MasterWorkerProtocol as an actual task owner.
+ this.client =
+ taskOwner.doAs(new PrivilegedExceptionAction<ProtoAsyncRpcClient>() {
+ @Override
+ public ProtoAsyncRpcClient run() throws Exception {
+ return new ProtoAsyncRpcClient(TajoWorkerProtocol.class, masterAddr);
+ }
+ });
+ this.master = client.getStub();
+
+ this.executionBlockId = executionBlockId;
+ this.queryId = executionBlockId.getQueryId();
+ this.nodeId = nodeId;
+ this.taskOwner = taskOwner;
+
+ this.taskRunnerContext = new TaskRunnerContext();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ this.queryConf = (QueryConf)conf;
+ try {
// initialize DFS and LocalFileSystems
- defaultFS = FileSystem.get(URI.create(conf.getVar(ConfVars.ROOT_DIR)),conf);
+ defaultFS = FileSystem.get(URI.create(queryConf.getVar(ConfVars.ROOT_DIR)),conf);
localFS = FileSystem.getLocal(conf);
// the base dir for an output dir
- baseDir = ConverterUtils.toString(appId)
- + "/output" + "/" + subQueryId.getId();
+ baseDir = queryId.toString()
+ + "/output" + "/" + executionBlockId.getId();
// initialize LocalDirAllocator
lDirAllocator = new LocalDirAllocator(ConfVars.TASK_LOCAL_DIR.varname);
@@ -139,9 +194,7 @@ public class TaskRunner extends AbstractService {
// Setup QueryEngine according to the query plan
// Here, we can setup row-based query engine or columnar query engine.
- this.queryEngine = new TajoQueryEngine(conf);
-
- Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+ this.queryEngine = new TajoQueryEngine(queryConf);
} catch (Throwable t) {
LOG.error(t);
}
@@ -152,39 +205,46 @@ public class TaskRunner extends AbstractService {
@Override
public void start() {
run();
+ super.start();
}
@Override
public void stop() {
- if (!isStopped()) {
- // If TaskRunner is stopped, all running or pending tasks will be marked as failed.
- for (Task task : tasks.values()) {
- if (task.getStatus() == TaskAttemptState.TA_PENDING ||
- task.getStatus() == TaskAttemptState.TA_RUNNING) {
- task.setState(TaskAttemptState.TA_FAILED);
- }
+ if(isStopped()) {
+ return;
+ }
+ // If this flag become true, taskLauncher will be terminated.
+ this.stopped = true;
+
+ // If TaskRunner is stopped, all running or pending tasks will be marked as failed.
+ for (Task task : tasks.values()) {
+ if (task.getStatus() == TaskAttemptState.TA_PENDING ||
+ task.getStatus() == TaskAttemptState.TA_RUNNING) {
+ task.setState(TaskAttemptState.TA_FAILED);
}
+ }
- // If this flag become true, taskLauncher will be terminated.
- this.stopped = true;
+ if(client != null) {
+ client.close();
+ client = null;
+ }
- LOG.info("STOPPED: " + nodeId);
- synchronized (this) {
- notifyAll();
- }
+ LOG.info("Stop TaskRunner: " + executionBlockId);
+ synchronized (this) {
+ notifyAll();
}
}
- class WorkerContext {
- public QueryConf getConf() {
- return conf;
+ public class TaskRunnerContext {
+ public QueryConf getQueryConf() {
+ return queryConf;
}
public String getNodeId() {
return nodeId.toString();
}
- public QueryMasterProtocolService.Interface getMaster() {
+ public TajoWorkerProtocolService.Interface getMaster() {
return master;
}
@@ -219,9 +279,17 @@ public class TaskRunner extends AbstractService {
public Path getBaseDir() {
return baseDirPath;
}
+
+ public ExecutionBlockId getExecutionBlockId() {
+ return executionBlockId;
+ }
+ }
+
+ public TaskRunnerContext getContext() {
+ return taskRunnerContext;
}
- static void fatalError(QueryMasterProtocolService.Interface proxy,
+ static void fatalError(TajoWorkerProtocolService.Interface proxy,
QueryUnitAttemptId taskAttemptId, String message) {
TaskFatalErrorReport.Builder builder = TaskFatalErrorReport.newBuilder()
.setId(taskAttemptId.getProto())
@@ -245,17 +313,27 @@ public class TaskRunner extends AbstractService {
try {
if (callFuture == null) {
callFuture = new CallFuture2<QueryUnitRequestProto>();
- master.getTask(null, ((ContainerIdPBImpl) containerId).getProto(),
- callFuture);
+ LOG.info("====>Request GetTask:" + executionBlockId + "," + containerId);
+ GetTaskRequestProto request = GetTaskRequestProto.newBuilder()
+ .setExecutionBlockId(executionBlockId.getProto())
+ .setContainerId(((ContainerIdPBImpl) containerId).getProto())
+ .build();
+ master.getTask(null, request, callFuture);
}
try {
// wait for an assigning task for 3 seconds
taskRequest = callFuture.get(3, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ if(stopped) {
+ break;
+ }
} catch (TimeoutException te) {
+ if(stopped) {
+ break;
+ }
// if there has been no assigning task for a given period,
// TaskRunner will retry to request an assigning task.
- LOG.error(te);
-
+ LOG.warn("Timeout getResource:" + executionBlockId + ", but retry", te);
continue;
}
@@ -264,9 +342,12 @@ public class TaskRunner extends AbstractService {
// If TaskRunner receives the terminal signal, TaskRunner will be terminated
// immediately.
if (taskRequest.getShouldDie()) {
- LOG.info("received ShouldDie flag");
+ LOG.info("Received ShouldDie flag:" + executionBlockId);
stop();
-
+ if(taskRunnerManager != null) {
+ //notify to TaskRunnerManager
+ taskRunnerManager.stopTask(executionBlockId);
+ }
} else {
LOG.info("Accumulated Received Task: " + (++receivedNum));
@@ -280,7 +361,7 @@ public class TaskRunner extends AbstractService {
LOG.info("Initializing: " + taskAttemptId);
Task task;
try {
- task = new Task(taskAttemptId, workerContext, master,
+ task = new Task(taskAttemptId, taskRunnerContext, master,
new QueryUnitRequestImpl(taskRequest));
tasks.put(taskAttemptId, task);
@@ -291,7 +372,7 @@ public class TaskRunner extends AbstractService {
// task.run() is a blocking call.
task.run();
} catch (Throwable t) {
- fatalError(workerContext.getMaster(), taskAttemptId, t.getMessage());
+ fatalError(taskRunnerContext.getMaster(), taskAttemptId, t.getMessage());
} finally {
callFuture = null;
taskRequest = null;
@@ -318,14 +399,6 @@ public class TaskRunner extends AbstractService {
}
}
- private class ShutdownHook implements Runnable {
- @Override
- public void run() {
- LOG.info("received SIGINT Signal");
- stop();
- }
- }
-
/**
* @return true if a stop has been requested.
*/
@@ -333,68 +406,7 @@ public class TaskRunner extends AbstractService {
return this.stopped;
}
- /**
- * TaskRunner takes 5 arguments as follows:
- * <ol>
- * <li>1st: SubQueryId</li>
- * <li>2nd: NodeId</li>
- * <li>3nd: ContainerId</li>
- * <li>4th: QueryMaster hostname</li>
- * <li>5th: QueryMaster port</li>
- * </ol>
- */
- public static void main(String[] args) throws Exception {
- // Restore QueryConf
- final QueryConf conf = new QueryConf();
- conf.addResource(new Path(QueryConf.FILENAME));
-
- LOG.info("MiniTajoYarn NM Local Dir: " + conf.get(ConfVars.TASK_LOCAL_DIR.varname));
- LOG.info("OUTPUT DIR: " + conf.getOutputPath());
- LOG.info("Tajo Root Dir: " + conf.getVar(ConfVars.ROOT_DIR));
-
- UserGroupInformation.setConfiguration(conf);
-
- // SubQueryId from String
- final SubQueryId subQueryId = TajoIdUtils.newSubQueryId(args[0]);
- // NodeId has a form of hostname:port.
- NodeId nodeId = ConverterUtils.toNodeId(args[1]);
- ContainerId containerId = ConverterUtils.toContainerId(args[2]);
-
- // QueryMaster's address
- String host = args[3];
- int port = Integer.parseInt(args[4]);
- final InetSocketAddress masterAddr =
- NetUtils.createSocketAddrForHost(host, port);
-
- // TODO - 'load credential' should be implemented
- // Getting taskOwner
- UserGroupInformation taskOwner =
- UserGroupInformation.createRemoteUser(conf.getVar(ConfVars.QUERY_USERNAME));
- //taskOwner.addToken(token);
-
- // QueryMasterService RPC
- ProtoAsyncRpcClient client;
- QueryMasterProtocolService.Interface master;
-
- // initialize MasterWorkerProtocol as an actual task owner.
- client =
- taskOwner.doAs(new PrivilegedExceptionAction<ProtoAsyncRpcClient>() {
- @Override
- public ProtoAsyncRpcClient run() throws Exception {
- return new ProtoAsyncRpcClient(QueryMasterProtocol.class, masterAddr);
- }
- });
- master = client.getStub();
-
-
- TaskRunner taskRunner = new TaskRunner(subQueryId, nodeId, taskOwner, master, containerId);
- try {
- taskRunner.init(conf);
- taskRunner.start();
- } finally {
- client.close();
- LOG.info("TaskRunner (" + nodeId + ") main thread exiting");
- System.exit(0);
- }
+ public ExecutionBlockId getExecutionBlockId() {
+ return this.executionBlockId;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
new file mode 100644
index 0000000..dcd44df
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TaskRunnerManager.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.conf.TajoConf;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TaskRunnerManager extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(TaskRunnerManager.class);
+
+ private Map<ExecutionBlockId, TaskRunner> taskRunnerMap = new HashMap<ExecutionBlockId, TaskRunner>();
+ private TajoWorker.WorkerContext workerContext;
+ private TajoConf tajoConf;
+ private AtomicBoolean stop = new AtomicBoolean(false);
+
+ public TaskRunnerManager(TajoWorker.WorkerContext workerContext) {
+ super(TaskRunnerManager.class.getName());
+
+ this.workerContext = workerContext;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ tajoConf = (TajoConf)conf;
+ super.init(tajoConf);
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(stop.get()) {
+ return;
+ }
+ stop.set(true);
+ synchronized(taskRunnerMap) {
+ for(TaskRunner eachTaskRunner: taskRunnerMap.values()) {
+ if(!eachTaskRunner.isStopped()) {
+ eachTaskRunner.stop();
+ }
+ }
+ }
+ super.stop();
+ if(!workerContext.isStandbyMode()) {
+ workerContext.stopWorker(true);
+ }
+ }
+
+ public void stopTask(ExecutionBlockId executionBlockId) {
+ LOG.info("Stop Task:" + executionBlockId);
+ synchronized(taskRunnerMap) {
+ taskRunnerMap.remove(executionBlockId);
+ }
+ if(!workerContext.isStandbyMode()) {
+ stop();
+ }
+ }
+
+ public void startTask(final String[] params) {
+ //TODO change to use event dispatcher
+ Thread t = new Thread() {
+ public void run() {
+ try {
+ QueryConf queryConf = new QueryConf(tajoConf);
+ TaskRunner taskRunner = new TaskRunner(TaskRunnerManager.this, queryConf, params);
+ synchronized(taskRunnerMap) {
+ taskRunnerMap.put(taskRunner.getContext().getExecutionBlockId(), taskRunner);
+ }
+ taskRunner.init(queryConf);
+ taskRunner.start();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ throw new RuntimeException(e.getMessage(), e);
+ }
+ }
+ };
+
+ t.start();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
new file mode 100644
index 0000000..9470a88
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/YarnResourceAllocator.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.client.YarnClient;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+import org.apache.tajo.master.TaskRunnerLauncher;
+import org.apache.tajo.master.YarnTaskRunnerLauncherImpl;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.rm.YarnRMContainerAllocator;
+
+public class YarnResourceAllocator extends AbstractResourceAllocator {
+ private YarnRMContainerAllocator rmAllocator;
+
+ private TaskRunnerLauncher taskRunnerLauncher;
+
+ private YarnRPC yarnRPC;
+
+ private YarnClient yarnClient;
+
+ private static final Log LOG = LogFactory.getLog(YarnResourceAllocator.class.getName());
+
+ private QueryMasterTask.QueryContext queryContext;
+
+ private QueryConf queryConf;
+
+ public YarnResourceAllocator(QueryMasterTask.QueryContext queryContext) {
+ this.queryContext = queryContext;
+ }
+
+ @Override
+ public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId) {
+ return new ContainerIdPBImpl(containerId);
+ }
+
+ @Override
+ public void allocateTaskWorker() {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ queryConf = (QueryConf)conf;
+
+ yarnRPC = YarnRPC.create(queryConf);
+
+ connectYarnClient();
+
+ taskRunnerLauncher = new YarnTaskRunnerLauncherImpl(queryContext, yarnRPC);
+ addService((org.apache.hadoop.yarn.service.Service) taskRunnerLauncher);
+ queryContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, taskRunnerLauncher);
+
+ rmAllocator = new YarnRMContainerAllocator(queryContext);
+ addService(rmAllocator);
+ queryContext.getDispatcher().register(ContainerAllocatorEventType.class, rmAllocator);
+ super.init(conf);
+ }
+
+ @Override
+ public void stop() {
+ try {
+ this.yarnClient.stop();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ super.stop();
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ private void connectYarnClient() {
+ this.yarnClient = new YarnClientImpl();
+ this.yarnClient.init(queryConf);
+ this.yarnClient.start();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
index d602d57..2ef0c4c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/dataserver/retriever/AdvancedDataRetriever.java
@@ -22,14 +22,14 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.util.TajoIdUtils;
import org.apache.tajo.worker.dataserver.FileAccessForbiddenException;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
import java.io.File;
import java.io.FileNotFoundException;
@@ -81,7 +81,7 @@ public class AdvancedDataRetriever implements DataRetriever {
List<String> qids = splitMaps(params.get("qid"));
for (String qid : qids) {
String[] ids = qid.split("_");
- SubQueryId suid = TajoIdUtils.newSubQueryId(params.get("sid").get(0));
+ ExecutionBlockId suid = TajoIdUtils.createExecutionBlockId(params.get("sid").get(0));
QueryUnitId quid = new QueryUnitId(suid, Integer.parseInt(ids[0]));
QueryUnitAttemptId attemptId = new QueryUnitAttemptId(quid,
Integer.parseInt(ids[1]));
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
index 61c14c4..43d99ef 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtocol.proto
@@ -51,13 +51,13 @@ message UpdateQueryResponse {
message SubmitQueryResponse {
required ResultCode resultCode = 1;
- optional ApplicationAttemptIdProto queryId = 2;
+ optional string queryId = 2;
optional string errorMessage = 3;
}
message GetQueryResultRequest {
optional SessionIdProto sessionId = 1;
- required ApplicationAttemptIdProto queryId = 2;
+ required string queryId = 2;
}
message GetQueryResultResponse {
@@ -70,7 +70,7 @@ message GetQueryListRequest {
}
message BriefQueryStatus {
- required ApplicationAttemptIdProto queryId = 1;
+ required string queryId = 1;
required QueryState state = 2;
required int32 executionTime = 3;
}
@@ -81,12 +81,12 @@ message GetQueryListResponse {
message GetQueryStatusRequest {
optional SessionIdProto sessionId = 1;
- required ApplicationAttemptIdProto queryId = 2;
+ required string queryId = 2;
}
message GetQueryStatusResponse {
required ResultCode resultCode = 1;
- required ApplicationAttemptIdProto queryId = 2;
+ required string queryId = 2;
optional QueryState state = 3;
optional float progress = 4;
optional int64 submitTime = 5;
@@ -142,7 +142,7 @@ service ClientProtocolService {
rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
rpc getQueryList(GetQueryListRequest) returns (GetQueryListResponse);
rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
- rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+ rpc killQuery(StringProto) returns (BoolProto);
rpc getClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse);
rpc existTable(StringProto) returns (BoolProto);
rpc getTableList(GetTableListRequest) returns (GetTableListResponse);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
index 2c5c2b6..f3b1005 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/ClientProtos.proto
@@ -51,13 +51,13 @@ message UpdateQueryResponse {
message SubmitQueryResponse {
required ResultCode resultCode = 1;
- optional ApplicationAttemptIdProto queryId = 2;
+ optional QueryIdProto queryId = 2;
optional string errorMessage = 3;
}
message GetQueryResultRequest {
optional SessionIdProto sessionId = 1;
- required ApplicationAttemptIdProto queryId = 2;
+ required QueryIdProto queryId = 2;
}
message GetQueryResultResponse {
@@ -70,7 +70,7 @@ message GetQueryListRequest {
}
message BriefQueryStatus {
- required ApplicationAttemptIdProto queryId = 1;
+ required QueryIdProto queryId = 1;
required QueryState state = 2;
required int32 executionTime = 3;
}
@@ -81,12 +81,12 @@ message GetQueryListResponse {
message GetQueryStatusRequest {
optional SessionIdProto sessionId = 1;
- required ApplicationAttemptIdProto queryId = 2;
+ required QueryIdProto queryId = 2;
}
message GetQueryStatusResponse {
required ResultCode resultCode = 1;
- required ApplicationAttemptIdProto queryId = 2;
+ required QueryIdProto queryId = 2;
optional QueryState state = 3;
optional float progress = 4;
optional int64 submitTime = 5;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
index 9337078..7da83bc 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterClientProtocol.proto
@@ -32,5 +32,5 @@ service QueryMasterClientProtocolService {
rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
- rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+ rpc killQuery(QueryIdProto) returns (BoolProto);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
deleted file mode 100644
index 08fc5c9..0000000
--- a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterManagerProtocol.proto
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tajo.ipc";
-option java_outer_classname = "QueryMasterManagerProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "yarn_protos.proto";
-import "tajo_protos.proto";
-import "TajoIdProtos.proto";
-import "CatalogProtos.proto";
-import "PrimitiveProtos.proto";
-
-message QueryHeartbeat {
- required ApplicationAttemptIdProto queryId = 1;
- required string queryMasterHost = 2;
- required int32 queryMasterPort = 3;
- required int32 queryMasterClientPort = 4;
- required QueryState state = 5;
- optional string statusMessage = 6;
-}
-
-message QueryHeartbeatResponse {
- message ResponseCommand {
- required string command = 1;
- repeated string params = 2;
- }
- required BoolProto heartbeatResult = 1;
- optional ResponseCommand responseCommand = 3;
-}
-
-service QueryMasterManagerProtocolService {
- rpc queryHeartbeat(QueryHeartbeat) returns (QueryHeartbeatResponse);
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
deleted file mode 100644
index b6a0602..0000000
--- a/tajo-core/tajo-core-backend/src/main/proto/QueryMasterProtocol.proto
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tajo.ipc";
-option java_outer_classname = "QueryMasterProtocol";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-import "yarn_protos.proto";
-import "tajo_protos.proto";
-import "TajoIdProtos.proto";
-import "CatalogProtos.proto";
-import "PrimitiveProtos.proto";
-
-message TaskStatusProto {
- required QueryUnitAttemptIdProto id = 1;
- required string workerName = 2;
- required float progress = 3;
- required TaskAttemptState state = 4;
- optional StatSetProto stats = 5;
- optional TableStatProto resultStats = 6;
- repeated Partition partitions = 7;
-}
-
-message TaskCompletionReport {
- required QueryUnitAttemptIdProto id = 1;
- optional StatSetProto stats = 2;
- optional TableStatProto resultStats = 3;
- repeated Partition partitions = 4;
-}
-
-message TaskFatalErrorReport {
- required QueryUnitAttemptIdProto id = 1;
- optional string error_message = 2;
-}
-
-message QueryUnitRequestProto {
- required QueryUnitAttemptIdProto id = 1;
- repeated FragmentProto fragments = 2;
- required string outputTable = 3;
- required bool clusteredOutput = 4;
- required string serializedData = 5;
- optional bool interQuery = 6 [default = false];
- repeated Fetch fetches = 7;
- optional bool shouldDie = 8;
-}
-
-message Fetch {
- required string name = 1;
- required string urls = 2;
-}
-
-message QueryUnitResponseProto {
- required string id = 1;
- required QueryState status = 2;
-}
-
-message StatusReportProto {
- required int64 timestamp = 1;
- required string serverName = 2;
- repeated TaskStatusProto status = 3;
- repeated QueryUnitAttemptIdProto pings = 4;
-}
-
-message CommandRequestProto {
- repeated Command command = 1;
-}
-
-message CommandResponseProto {
-}
-
-message Command {
- required QueryUnitAttemptIdProto id = 1;
- required CommandType type = 2;
-}
-
-enum CommandType {
- PREPARE = 0;
- LAUNCH = 1;
- STOP = 2;
- FINALIZE = 3;
-}
-
-message Partition {
- required int32 partitionKey = 1;
- optional string fileName = 2;
-}
-
-message ServerStatusProto {
- message System {
- required int32 availableProcessors = 1;
- required int64 freeMemory = 2;
- required int64 maxMemory = 3;
- required int64 totalMemory = 4;
- }
- message Disk {
- required string absolutePath = 1;
- required int64 totalSpace = 2;
- required int64 freeSpace = 3;
- required int64 usableSpace = 4;
- }
- required System system = 1;
- repeated Disk disk = 2;
- required int32 taskNum = 3;
-}
-
-service QueryMasterProtocolService {
- //from Worker
- rpc getTask(ContainerIdProto) returns (QueryUnitRequestProto);
- rpc statusUpdate (TaskStatusProto) returns (BoolProto);
- rpc ping (QueryUnitAttemptIdProto) returns (BoolProto);
- rpc fatalError(TaskFatalErrorReport) returns (BoolProto);
- rpc done (TaskCompletionReport) returns (BoolProto);
-
- //from QueryMasterManager
- rpc executeQuery(StringProto) returns (BoolProto);
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
index 04c67f2..a87c825 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoIdProtos.proto
@@ -21,15 +21,18 @@ option java_outer_classname = "TajoIdProtos";
option java_generic_services = false;
option java_generate_equals_and_hash = true;
-import "yarn_protos.proto";
+message QueryIdProto {
+ required string id = 1;
+ required int32 seq = 2;
+}
-message SubQueryIdProto {
- required ApplicationAttemptIdProto queryId = 1;
+message ExecutionBlockIdProto {
+ required QueryIdProto queryId = 1;
required int32 id = 2;
}
message QueryUnitIdProto {
- required SubQueryIdProto subQueryId = 1;
+ required ExecutionBlockIdProto executionBlockId = 1;
required int32 id = 2;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
index ef7e711..26dbbed 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterClientProtocol.proto
@@ -16,6 +16,7 @@
* limitations under the License.
*/
+//TajoClient -> TajoMaster Protocol
option java_package = "org.apache.tajo.ipc";
option java_outer_classname = "TajoMasterClientProtocol";
option java_generic_services = true;
@@ -30,12 +31,12 @@ import "ClientProtos.proto";
service TajoMasterClientProtocolService {
rpc updateSessionVariables(UpdateSessionVariableRequest) returns (BoolProto);
- rpc submitQuery(QueryRequest) returns (SubmitQueryResponse);
+ rpc submitQuery(QueryRequest) returns (GetQueryStatusResponse);
rpc updateQuery(QueryRequest) returns (UpdateQueryResponse);
rpc getQueryResult(GetQueryResultRequest) returns (GetQueryResultResponse);
rpc getQueryList(GetQueryListRequest) returns (GetQueryListResponse);
rpc getQueryStatus(GetQueryStatusRequest) returns (GetQueryStatusResponse);
- rpc killQuery(ApplicationAttemptIdProto) returns (BoolProto);
+ rpc killQuery(QueryIdProto) returns (BoolProto);
rpc getClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse);
rpc existTable(StringProto) returns (BoolProto);
rpc getTableList(GetTableListRequest) returns (GetTableListResponse);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
new file mode 100644
index 0000000..0153c8d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoMasterProtocol.proto
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+//TajoWorker -> TajoMaster protocol
+
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoMasterProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+
+message ServerStatusProto {
+ message System {
+ required int32 availableProcessors = 1;
+ required int32 freeMemoryMB = 2;
+ required int32 maxMemoryMB = 3;
+ required int32 totalMemoryMB = 4;
+ }
+ message Disk {
+ required string absolutePath = 1;
+ required int64 totalSpace = 2;
+ required int64 freeSpace = 3;
+ required int64 usableSpace = 4;
+ }
+ required System system = 1;
+ required int32 diskSlots = 2;
+ repeated Disk disk = 3;
+ required int32 runningTaskNum = 4;
+}
+
+message TajoHeartbeat {
+ required string tajoWorkerHost = 1;
+ required int32 tajoWorkerPort = 2;
+ optional ServerStatusProto serverStatus = 3;
+ optional int32 tajoWorkerClientPort = 4;
+ optional QueryIdProto queryId = 5;
+ optional QueryState state = 6;
+ optional string statusMessage = 7;
+}
+
+message TajoHeartbeatResponse {
+ message ResponseCommand {
+ required string command = 1;
+ repeated string params = 2;
+ }
+ required BoolProto heartbeatResult = 1;
+ optional ResponseCommand responseCommand = 3;
+}
+
+message WorkerResourceAllocationRequest {
+ required ExecutionBlockIdProto executionBlockId = 1;
+ required int32 numWorks = 2;
+ required int32 memoryMBSlots = 3 ;
+ required int32 diskSlots = 4;
+}
+
+message WorkerResourceProto {
+ required string workerHostAndPort = 1;
+ required ExecutionBlockIdProto executionBlockId = 2;
+ required int32 memoryMBSlots = 3 ;
+ required int32 diskSlots = 4;
+}
+
+message WorkerResourceReleaseRequest {
+ repeated WorkerResourceProto workerResources = 1;
+}
+
+message WorkerResourceAllocationResponse {
+ required ExecutionBlockIdProto executionBlockId = 1;
+ repeated string allocatedWorks = 2;
+}
+
+service TajoMasterProtocolService {
+ rpc heartbeat(TajoHeartbeat) returns (TajoHeartbeatResponse);
+ rpc allocateWorkerResources(WorkerResourceAllocationRequest) returns (WorkerResourceAllocationResponse);
+ rpc releaseWorkerResource(WorkerResourceReleaseRequest) returns (BoolProto);
+ rpc stopQueryMaster(QueryIdProto) returns (BoolProto);
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
new file mode 100644
index 0000000..88a2029
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// TajoMaster -> TajoWorker, TajoWorker(QueryMaster) <-> TajoWorker Protocol
+option java_package = "org.apache.tajo.ipc";
+option java_outer_classname = "TajoWorkerProtocol";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+import "yarn_protos.proto";
+import "tajo_protos.proto";
+import "TajoIdProtos.proto";
+import "CatalogProtos.proto";
+import "PrimitiveProtos.proto";
+
+message TaskStatusProto {
+ required QueryUnitAttemptIdProto id = 1;
+ required string workerName = 2;
+ required float progress = 3;
+ required TaskAttemptState state = 4;
+ optional StatSetProto stats = 5;
+ optional TableStatProto resultStats = 6;
+ repeated Partition partitions = 7;
+}
+
+message TaskCompletionReport {
+ required QueryUnitAttemptIdProto id = 1;
+ optional StatSetProto stats = 2;
+ optional TableStatProto resultStats = 3;
+ repeated Partition partitions = 4;
+}
+
+message TaskFatalErrorReport {
+ required QueryUnitAttemptIdProto id = 1;
+ optional string error_message = 2;
+}
+
+message QueryUnitRequestProto {
+ required QueryUnitAttemptIdProto id = 1;
+ repeated FragmentProto fragments = 2;
+ required string outputTable = 3;
+ required bool clusteredOutput = 4;
+ required string serializedData = 5;
+ optional bool interQuery = 6 [default = false];
+ repeated Fetch fetches = 7;
+ optional bool shouldDie = 8;
+}
+
+message Fetch {
+ required string name = 1;
+ required string urls = 2;
+}
+
+message QueryUnitResponseProto {
+ required string id = 1;
+ required QueryState status = 2;
+}
+
+message StatusReportProto {
+ required int64 timestamp = 1;
+ required string serverName = 2;
+ repeated TaskStatusProto status = 3;
+ repeated QueryUnitAttemptIdProto pings = 4;
+}
+
+message CommandRequestProto {
+ repeated Command command = 1;
+}
+
+message CommandResponseProto {
+}
+
+message Command {
+ required QueryUnitAttemptIdProto id = 1;
+ required CommandType type = 2;
+}
+
+enum CommandType {
+ PREPARE = 0;
+ LAUNCH = 1;
+ STOP = 2;
+ FINALIZE = 3;
+}
+
+message Partition {
+ required int32 partitionKey = 1;
+ optional string fileName = 2;
+}
+
+message QueryExecutionRequestProto {
+ required QueryIdProto queryId = 1;
+ required StringProto logicalPlanJson = 2;
+}
+
+message GetTaskRequestProto {
+ required ContainerIdProto containerId = 1;
+ required ExecutionBlockIdProto executionBlockId = 2;
+}
+
+message RunExecutionBlockRequestProto {
+ required string executionBlockId = 1;
+ required string queryMasterHost = 2;
+ required int32 queryMasterPort = 3;
+ required string nodeId = 4;
+ required string containerId = 5;
+ optional string queryOutputPath = 6;
+}
+
+service TajoWorkerProtocolService {
+ //from Worker
+ rpc getTask(GetTaskRequestProto) returns (QueryUnitRequestProto);
+ rpc statusUpdate (TaskStatusProto) returns (BoolProto);
+ rpc ping (QueryUnitAttemptIdProto) returns (BoolProto);
+ rpc fatalError(TaskFatalErrorReport) returns (BoolProto);
+ rpc done (TaskCompletionReport) returns (BoolProto);
+
+ //from TajoMaster's QueryJobManager
+ rpc executeQuery(QueryExecutionRequestProto) returns (BoolProto);
+
+ //from QueryMaster(Worker)
+ rpc executeExecutionBlock(RunExecutionBlockRequestProto) returns (BoolProto);
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/log4j.properties b/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
index 2b42975..007c8f5 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/main/resources/log4j.properties
@@ -23,3 +23,6 @@ log4j.threshhold=INFO
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml b/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
index a1a111c..e8ad503 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
+++ b/tajo-core/tajo-core-backend/src/main/resources/tajo-default.xml
@@ -43,4 +43,72 @@
</property>
+
+ <property>
+ <name>tajo.master.clientservice.addr</name>
+ <value>127.0.0.1:9004</value>
+ </property>
+
+ <property>
+ <name>tajo.master.manager.addr</name>
+ <value>127.0.0.1:9005</value>
+ <description>rpc port for tajo worker</description>
+ </property>
+
+ <property>
+ <name>tajo.query.session.timeout</name>
+ <value>60000</value>
+ <description>ms</description>
+ </property>
+
+ <property>
+ <name>tajo.resource.manager</name>
+ <value>org.apache.tajo.master.rm.TajoWorkerResourceManager</value>
+ <description>This can be org.apache.tajo.master.rm.TajoWorkerResourceManager or org.apache.tajo.master.rm.YarnTajoResourceManager</description>
+ </property>
+
+ <property>
+ <name>tajo.querymaster.memoryMB</name>
+ <value>512</value>
+ <description>the memory slot size for a QeuryMaster</description>
+ </property>
+
+ <property>
+ <name>tajo.worker.slots.use.os.info</name>
+ <value>true</value>
+ <description>If true, Tajo system obtains the physical resource information from OS.
+ If false, the physical resource information is obtained from the below configs.</description>
+ </property>
+
+ <!-- Default Node's Physical information -->
+ <!-- The below configs are used if tajo.worker.slots.use.os.info is set to true. -->
+ <property>
+ <name>tajo.worker.slots.os.memory.ratio</name>
+ <value>0.8f</value>
+ <description>The ratio of allocatable memory to the total system memory</description>
+ </property>
+
+ <property>
+ <name>tajo.worker.slots.memoryMB</name>
+ <value>2048</value>
+ <description></description>
+ </property>
+
+ <property>
+ <name>tajo.worker.slots.disk</name>
+ <value>2</value>
+ <description>The number of disks on a worker</description>
+ </property>
+
+ <property>
+ <name>tajo.worker.slots.disk.concurrency</name>
+ <value>4</value>
+ <description>the maximum concurrency number per disk slot</description>
+ </property>
+
+ <property>
+ <name>tajo.worker.slots.cpu.core</name>
+ <value>4</value>
+ <description>The number of CPU cores on a worker</description>
+ </property>
</configuration>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/log4j.properties b/tajo-core/tajo-core-backend/src/test/java/log4j.properties
index c1ac487..749124c 100644
--- a/tajo-core/tajo-core-backend/src/test/java/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/test/java/log4j.properties
@@ -23,3 +23,6 @@ log4j.threshhold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
index 5e8d11d..1667813 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -23,7 +23,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Options;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
import org.apache.tajo.catalog.proto.CatalogProtos;
import org.apache.tajo.client.TajoClient;
import org.apache.tajo.conf.TajoConf;
@@ -83,7 +86,11 @@ public class LocalTajoTestingUtility {
}
public void shutdown() throws IOException {
- client.close();
- util.shutdownMiniCluster();
+ if(client != null) {
+ client.close();
+ }
+ if(util != null) {
+ util.shutdownMiniCluster();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
index 5b7267f..37e2721 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/MiniTajoYarnCluster.java
@@ -110,9 +110,6 @@ public class MiniTajoYarnCluster extends MiniYARNCluster {
conf.setInt("yarn.nodemanager.delete.debug-delay-sec", 600);
- // Disable virtual memory constraints for containers
- conf.setBoolean("yarn.nodemanager.vmem-check-enabled", false);
-
super.init(conf);
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 32b1f56..041043d 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -27,9 +27,8 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.proto.CatalogProtos;
@@ -37,12 +36,17 @@ import org.apache.tajo.client.TajoClient;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.TajoWorker;
import java.io.*;
+import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.URL;
import java.sql.ResultSet;
+import java.util.ArrayList;
+import java.util.List;
import java.util.UUID;
public class TajoTestingCluster {
@@ -54,8 +58,9 @@ public class TajoTestingCluster {
private MiniDFSCluster dfsCluster;
private MiniCatalogServer catalogServer;
-
private TajoMaster tajoMaster;
+ private List<TajoWorker> tajoWorkers = new ArrayList<TajoWorker>();
+ private boolean standbyWorkerMode = false;
// If non-null, then already a cluster running.
private File clusterTestBuildDir = null;
@@ -73,7 +78,10 @@ public class TajoTestingCluster {
public static final String DEFAULT_TEST_DIRECTORY = "target/test-data";
public TajoTestingCluster() {
- this.conf = new TajoConf();
+ this.conf = new TajoConf();
+ this.standbyWorkerMode =
+ this.conf.get("tajo.resource.manager", TajoWorkerResourceManager.class.getCanonicalName())
+ .indexOf(TajoWorkerResourceManager.class.getName()) >= 0;
}
public TajoConf getConfiguration() {
@@ -113,7 +121,7 @@ public class TajoTestingCluster {
String dirStr = getTestDir(randomStr).toString();
File dir = new File(dirStr).getAbsoluteFile();
// Have it cleaned up on exit
- dir.deleteOnExit();
+ //dir.deleteOnExit();
return dir;
}
@@ -155,8 +163,7 @@ public class TajoTestingCluster {
System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
this.clusterTestBuildDir.toString());
- conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
- MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(new HdfsConfiguration(conf));
+ MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
builder.hosts(hosts);
builder.numDataNodes(servers);
builder.format(true);
@@ -210,7 +217,7 @@ public class TajoTestingCluster {
catalogServer = new MiniCatalogServer(conf);
CatalogServer catServer = catalogServer.getCatalogServer();
InetSocketAddress sockAddr = catServer.getBindAddress();
- c.setVar(ConfVars.CATALOG_ADDRESS, NetUtils.normalizeInetSocketAddress(sockAddr));
+ c.setVar(ConfVars.CATALOG_ADDRESS, NetUtils.getIpPortString(sockAddr));
return this.catalogServer;
}
@@ -232,11 +239,12 @@ public class TajoTestingCluster {
TajoConf c = getConfiguration();
c.setVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS, "localhost:0");
c.setVar(ConfVars.CLIENT_SERVICE_ADDRESS, "localhost:0");
- c.setVar(ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS, "localhost:0");
+ c.setVar(ConfVars.TAJO_MASTER_SERVICE_ADDRESS, "localhost:0");
c.setVar(ConfVars.CATALOG_ADDRESS, "localhost:0");
c.set(CatalogConstants.STORE_CLASS, "org.apache.tajo.catalog.store.MemStore");
c.set(CatalogConstants.JDBC_URI, "jdbc:derby:target/test-data/tcat/db");
+
LOG.info("derby repository is set to "+conf.get(CatalogConstants.JDBC_URI));
if (!local) {
@@ -253,11 +261,38 @@ public class TajoTestingCluster {
this.conf.setVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS, c.getVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS));
this.conf.setVar(ConfVars.CLIENT_SERVICE_ADDRESS, c.getVar(ConfVars.CLIENT_SERVICE_ADDRESS));
+
+ InetSocketAddress tajoMasterAddress = tajoMaster.getContext().getTajoMasterService().getBindAddress();
+
+ this.conf.setVar(ConfVars.TAJO_MASTER_SERVICE_ADDRESS,
+ tajoMasterAddress.getHostName() + ":" + tajoMasterAddress.getPort());
+
this.conf.setVar(ConfVars.CATALOG_ADDRESS, c.getVar(ConfVars.CATALOG_ADDRESS));
+ if(standbyWorkerMode) {
+ startTajoWorkers(numSlaves);
+ }
LOG.info("Mini Tajo cluster is up");
}
+ private void startTajoWorkers(int numSlaves) throws Exception {
+ for(int i = 0; i < 1; i++) {
+ TajoWorker tajoWorker = new TajoWorker("all");
+
+ TajoConf workerConf = new TajoConf(this.conf);
+
+ workerConf.setInt("tajo.worker.info.port", 0);
+ workerConf.setInt("tajo.worker.client.rpc.port", 0);
+ workerConf.setInt("tajo.worker.manager.rpc.port", 0);
+ workerConf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname, 0);
+
+ tajoWorker.startWorker(workerConf, new String[]{"standby"});
+
+ LOG.info("=====> MiniTajoCluster Worker #" + (i + 1) + " started.");
+ tajoWorkers.add(tajoWorker);
+ }
+ }
+
public void restartTajoCluster(int numSlaves) throws Exception {
tajoMaster.stop();
tajoMaster.start();
@@ -273,6 +308,10 @@ public class TajoTestingCluster {
if(this.tajoMaster != null) {
this.tajoMaster.stop();
}
+ for(TajoWorker eachWorker: tajoWorkers) {
+ eachWorker.stopWorkerForce();
+ }
+ tajoWorkers.clear();
this.tajoMaster= null;
}
@@ -297,7 +336,8 @@ public class TajoTestingCluster {
*/
public void startMiniCluster(final int numSlaves)
throws Exception {
- startMiniCluster(numSlaves, null);
+ String localHostName = InetAddress.getLocalHost().getHostName();
+ startMiniCluster(numSlaves, new String[] {localHostName});
}
public void startMiniCluster(final int numSlaves,
@@ -331,17 +371,19 @@ public class TajoTestingCluster {
startMiniDFSCluster(numDataNodes, this.clusterTestBuildDir, dataNodeHosts);
this.dfsCluster.waitClusterUp();
+ if(!standbyWorkerMode) {
+ startMiniYarnCluster();
+ }
+
+ startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, false);
+ }
+
+ private void startMiniYarnCluster() throws Exception {
LOG.info("Starting up YARN cluster");
// Scheduler properties required for YARN to work
conf.set("yarn.scheduler.capacity.root.queues", "default");
conf.set("yarn.scheduler.capacity.root.default.capacity", "100");
- // fixed thread OOM
- conf.setInt(YarnConfiguration.RM_CLIENT_THREAD_COUNT, 2);
- conf.setInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT, 2);
- conf.setInt(YarnConfiguration.RM_RESOURCE_TRACKER_CLIENT_THREAD_COUNT, 2);
- conf.setInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT, 2);
-
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 384);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 3000);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, 1);
@@ -368,8 +410,6 @@ public class TajoTestingCluster {
yarnCluster.getConfig().writeXml(os);
os.close();
}
-
- startMiniTajoCluster(this.clusterTestBuildDir, numSlaves, false);
}
public void startMiniClusterInLocal(final int numSlaves) throws Exception {
@@ -413,9 +453,12 @@ public class TajoTestingCluster {
}
if(this.clusterTestBuildDir != null && this.clusterTestBuildDir.exists()) {
- LocalFileSystem localFS = LocalFileSystem.getLocal(conf);
- localFS.delete(
- new Path(clusterTestBuildDir.toString()), true);
+ if(!ShutdownHookManager.get().isShutdownInProgress()) {
+ //TODO clean test dir when ShutdownInProgress
+ LocalFileSystem localFS = LocalFileSystem.getLocal(conf);
+ localFS.delete(
+ new Path(clusterTestBuildDir.toString()), true);
+ }
this.clusterTestBuildDir = null;
}
@@ -457,6 +500,12 @@ public class TajoTestingCluster {
String query) throws Exception {
TpchTestBase instance = TpchTestBase.getInstance();
TajoTestingCluster util = instance.getTestingCluster();
+ while(true) {
+ if(util.getMaster().isMasterRunning()) {
+ break;
+ }
+ Thread.sleep(1000);
+ }
TajoConf conf = util.getConfiguration();
TajoClient client = new TajoClient(conf);
@@ -503,33 +552,4 @@ public class TajoTestingCluster {
Closeables.closeQuietly(writer);
}
}
-
-
- /**
- * @param args
- * @throws Exception
- */
- public static void main(String[] args) throws Exception {
- TajoTestingCluster cluster = new TajoTestingCluster();
- File f = cluster.setupClusterTestBuildDir();
- System.out.println("first setupClusterTestBuildDir: " + f);
- f = cluster.setupClusterTestBuildDir();
- System.out.println("second setupClusterTestBuildDir: " + f);
- f = cluster.getTestDir();
- System.out.println("getTestDir() after second: " + f);
- f = cluster.getTestDir("abc");
- System.out.println("getTestDir(\"abc\") after second: " + f);
-
- cluster.initTestDir();
- f = cluster.getTestDir();
- System.out.println("getTestDir() after initTestDir: " + f);
- f = cluster.getTestDir("abc");
- System.out.println("getTestDir(\"abc\") after initTestDir: " + f);
- f = cluster.setupClusterTestBuildDir();
- System.out.println("setupClusterTestBuildDir() after initTestDir: " + f);
-
- TajoTestingCluster cluster2 = new TajoTestingCluster();
- File f2 = cluster2.setupClusterTestBuildDir();
- System.out.println("first setupClusterTestBuildDir of cluster2: " + f2);
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
index b4d920f..7b82952 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
@@ -27,7 +27,6 @@ public class TestQueryIdFactory {
@Before
public void setup() {
- QueryIdFactory.reset();
}
@Test
@@ -40,15 +39,15 @@ public class TestQueryIdFactory {
@Test
public void testNewSubQueryId() {
QueryId qid = QueryIdFactory.newQueryId();
- SubQueryId subqid1 = QueryIdFactory.newSubQueryId(qid);
- SubQueryId subqid2 = QueryIdFactory.newSubQueryId(qid);
+ ExecutionBlockId subqid1 = QueryIdFactory.newExecutionBlockId(qid);
+ ExecutionBlockId subqid2 = QueryIdFactory.newExecutionBlockId(qid);
assertTrue(subqid1.compareTo(subqid2) < 0);
}
@Test
public void testNewQueryUnitId() {
QueryId qid = QueryIdFactory.newQueryId();
- SubQueryId subid = QueryIdFactory.newSubQueryId(qid);
+ ExecutionBlockId subid = QueryIdFactory.newExecutionBlockId(qid);
QueryUnitId quid1 = QueryIdFactory.newQueryUnitId(subid);
QueryUnitId quid2 = QueryIdFactory.newQueryUnitId(subid);
assertTrue(quid1.compareTo(quid2) < 0);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
index 386fe02..1997159 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
@@ -20,8 +20,8 @@ package org.apache.tajo;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.junit.Test;
import org.apache.tajo.util.TajoIdUtils;
+import org.junit.Test;
import static org.junit.Assert.*;
@@ -31,10 +31,10 @@ public class TestTajoIds {
long ts1 = 1315890136000l;
long ts2 = 1315890136001l;
- QueryId j1 = createQueryId(ts1, 2, 1);
- QueryId j2 = createQueryId(ts1, 1, 2);
- QueryId j3 = createQueryId(ts2, 1, 2);
- QueryId j4 = createQueryId(ts1, 2, 1);
+ QueryId j1 = createQueryId(ts1, 2);
+ QueryId j2 = createQueryId(ts1, 1);
+ QueryId j3 = createQueryId(ts2, 1);
+ QueryId j4 = createQueryId(ts1, 2);
assertTrue(j1.equals(j4));
assertFalse(j1.equals(j2));
@@ -48,42 +48,42 @@ public class TestTajoIds {
assertFalse(j1.hashCode() == j2.hashCode());
assertFalse(j1.hashCode() == j3.hashCode());
- QueryId j5 = createQueryId(ts1, 231415, 2);
- assertEquals("q_" + ts1 + "_0002_000001", j1.toString());
- assertEquals("q_" + ts1 + "_231415_000002", j5.toString());
+ QueryId j5 = createQueryId(ts1, 231415);
+ assertEquals("q_" + ts1 + "_0002", j1.toString());
+ assertEquals("q_" + ts1 + "_231415", j5.toString());
}
@Test
public void testQueryIds() {
long timeId = 1315890136000l;
- QueryId queryId = createQueryId(timeId, 1, 1);
- assertEquals("q_" + timeId + "_0001_000001", queryId.toString());
+ QueryId queryId = createQueryId(timeId, 1);
+ assertEquals("q_" + timeId + "_0001", queryId.toString());
- SubQueryId subId = TajoIdUtils.newSubQueryId(queryId, 2);
- assertEquals("sq_" + timeId +"_0001_000001_02", subId.toString());
+ ExecutionBlockId subId = QueryIdFactory.newExecutionBlockId(queryId, 2);
+ assertEquals("eb_" + timeId +"_0001_000002", subId.toString());
QueryUnitId qId = new QueryUnitId(subId, 5);
- assertEquals("t_" + timeId + "_0001_000001_02_000005", qId.toString());
+ assertEquals("t_" + timeId + "_0001_000002_000005", qId.toString());
QueryUnitAttemptId attemptId = new QueryUnitAttemptId(qId, 4);
- assertEquals("ta_" + timeId + "_0001_000001_02_000005_04", attemptId.toString());
+ assertEquals("ta_" + timeId + "_0001_000002_000005_04", attemptId.toString());
}
@Test
public void testEqualsObject() {
long timeId = System.currentTimeMillis();
- QueryId queryId1 = createQueryId(timeId, 1, 1);
- QueryId queryId2 = createQueryId(timeId, 2, 2);
+ QueryId queryId1 = createQueryId(timeId, 1);
+ QueryId queryId2 = createQueryId(timeId, 2);
assertNotSame(queryId1, queryId2);
- QueryId queryId3 = createQueryId(timeId, 1, 1);
+ QueryId queryId3 = createQueryId(timeId, 1);
assertEquals(queryId1, queryId3);
- SubQueryId sid1 = TajoIdUtils.newSubQueryId(queryId1, 1);
- SubQueryId sid2 = TajoIdUtils.newSubQueryId(queryId1, 2);
+ ExecutionBlockId sid1 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+ ExecutionBlockId sid2 = QueryIdFactory.newExecutionBlockId(queryId1, 2);
assertNotSame(sid1, sid2);
- SubQueryId sid3 = TajoIdUtils.newSubQueryId(queryId1, 1);
+ ExecutionBlockId sid3 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
assertEquals(sid1, sid3);
QueryUnitId qid1 = new QueryUnitId(sid1, 9);
@@ -97,16 +97,16 @@ public class TestTajoIds {
public void testCompareTo() {
long time = System.currentTimeMillis();
- QueryId queryId1 = createQueryId(time, 1, 1);
- QueryId queryId2 = createQueryId(time, 2, 2);
- QueryId queryId3 = createQueryId(time, 1, 1);
+ QueryId queryId1 = createQueryId(time, 1);
+ QueryId queryId2 = createQueryId(time, 2);
+ QueryId queryId3 = createQueryId(time, 1);
assertEquals(-1, queryId1.compareTo(queryId2));
assertEquals(1, queryId2.compareTo(queryId1));
assertEquals(0, queryId3.compareTo(queryId1));
-
- SubQueryId sid1 = TajoIdUtils.newSubQueryId(queryId1, 1);
- SubQueryId sid2 = TajoIdUtils.newSubQueryId(queryId1, 2);
- SubQueryId sid3 = TajoIdUtils.newSubQueryId(queryId1, 1);
+
+ ExecutionBlockId sid1 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
+ ExecutionBlockId sid2 = QueryIdFactory.newExecutionBlockId(queryId1, 2);
+ ExecutionBlockId sid3 = QueryIdFactory.newExecutionBlockId(queryId1, 1);
assertEquals(-1, sid1.compareTo(sid2));
assertEquals(1, sid2.compareTo(sid1));
assertEquals(0, sid3.compareTo(sid1));
@@ -121,33 +121,33 @@ public class TestTajoIds {
@Test
public void testConstructFromString() {
- QueryIdFactory.reset();
+// QueryIdFactory.reset();
QueryId qid1 = QueryIdFactory.newQueryId();
- QueryId qid2 = TajoIdUtils.createQueryId(qid1.toString());
+ QueryId qid2 = TajoIdUtils.parseQueryId(qid1.toString());
assertEquals(qid1, qid2);
-
- SubQueryId sub1 = QueryIdFactory.newSubQueryId(qid1);
- SubQueryId sub2 = TajoIdUtils.newSubQueryId(sub1.toString());
+
+ ExecutionBlockId sub1 = QueryIdFactory.newExecutionBlockId(qid1);
+ ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
assertEquals(sub1, sub2);
QueryUnitId u1 = QueryIdFactory.newQueryUnitId(sub1);
- QueryUnitId u2 = new QueryUnitId(u1.toString());
+ QueryUnitId u2 = new QueryUnitId(u1.getProto());
assertEquals(u1, u2);
QueryUnitAttemptId attempt1 = new QueryUnitAttemptId(u1, 1);
- QueryUnitAttemptId attempt2 = new QueryUnitAttemptId(attempt1.toString());
+ QueryUnitAttemptId attempt2 = new QueryUnitAttemptId(attempt1.getProto());
assertEquals(attempt1, attempt2);
}
@Test
public void testConstructFromPB() {
- QueryIdFactory.reset();
+// QueryIdFactory.reset();
QueryId qid1 = QueryIdFactory.newQueryId();
QueryId qid2 = new QueryId(qid1.getProto());
assertEquals(qid1, qid2);
- SubQueryId sub1 = QueryIdFactory.newSubQueryId(qid1);
- SubQueryId sub2 = new SubQueryId(sub1.getProto());
+ ExecutionBlockId sub1 = QueryIdFactory.newExecutionBlockId(qid1);
+ ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
assertEquals(sub1, sub2);
QueryUnitId u1 = QueryIdFactory.newQueryUnitId(sub1);
@@ -159,9 +159,9 @@ public class TestTajoIds {
assertEquals(attempt1, attempt2);
}
- public static QueryId createQueryId(long timestamp, int id, int attemptId) {
+ public static QueryId createQueryId(long timestamp, int id) {
ApplicationId appId = BuilderUtils.newApplicationId(timestamp, id);
- return TajoIdUtils.createQueryId(appId, attemptId);
+ return QueryIdFactory.newQueryId(appId.toString());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
index c761103..ad3d676 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TpchTestBase.java
@@ -49,7 +49,6 @@ public class TpchTestBase {
try {
testBase = new TpchTestBase();
testBase.setUp();
- Runtime.getRuntime().addShutdownHook(new ShutdownHook());
} catch (Exception e) {
LOG.error(e.getMessage(), e);
}
@@ -107,19 +106,11 @@ public class TpchTestBase {
return util.getTestingCluster();
}
- public static class ShutdownHook extends Thread {
-
- @Override
- public void run() {
- try {
- testBase.tearDown();
- } catch (IOException e) {
- LOG.error(e);
- }
+ public void tearDown() throws IOException {
+ try {
+ Thread.sleep(2000);
+ } catch (InterruptedException e) {
}
- }
-
- private void tearDown() throws IOException {
util.shutdown();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
index cc75726..ede73c5 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
@@ -136,7 +136,6 @@ public class TestGlobalQueryPlanner {
catalog.addTable(desc);
}
- QueryIdFactory.reset();
queryId = QueryIdFactory.newQueryId();
dispatcher.stop();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
index 4455763..c665b44 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
@@ -16,9 +16,6 @@
* limitations under the License.
*/
-/**
- *
- */
package org.apache.tajo.engine.planner.global;
import org.apache.hadoop.fs.FileSystem;
@@ -128,11 +125,11 @@ public class TestGlobalQueryOptimizer {
catalog.addTable(desc);
}
- QueryIdFactory.reset();
+ //QueryIdFactory.reset();
queryId = QueryIdFactory.newQueryId();
optimizer = new GlobalOptimizer();
}
-
+
@AfterClass
public static void terminate() throws IOException {
util.shutdownCatalogCluster();
@@ -147,7 +144,7 @@ public class TestGlobalQueryOptimizer {
MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
globalPlan = optimizer.optimize(globalPlan);
-
+
ExecutionBlock unit = globalPlan.getRoot();
StoreTableNode store = unit.getStoreTableNode();
assertEquals(NodeType.PROJECTION, store.getChild().getType());
@@ -156,14 +153,14 @@ public class TestGlobalQueryOptimizer {
SortNode sort = (SortNode) proj.getChild();
assertEquals(NodeType.SCAN, sort.getChild().getType());
ScanNode scan = (ScanNode) sort.getChild();
-
+
assertTrue(unit.hasChildBlock());
unit = unit.getChildBlock(scan);
store = unit.getStoreTableNode();
assertEquals(NodeType.SORT, store.getChild().getType());
sort = (SortNode) store.getChild();
assertEquals(NodeType.JOIN, sort.getChild().getType());
-
+
assertTrue(unit.hasChildBlock());
for (ScanNode prevscan : unit.getScanNodes()) {
ExecutionBlock prev = unit.getChildBlock(prevscan);
@@ -171,4 +168,4 @@ public class TestGlobalQueryOptimizer {
assertEquals(NodeType.SCAN, store.getChild().getType());
}
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index dbb3862..75e3b1e 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -25,7 +25,6 @@ import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
@@ -78,7 +77,6 @@ public class TestPhysicalPlanner {
@BeforeClass
public static void setUp() throws Exception {
- QueryIdFactory.reset();
util = new TajoTestingCluster();
util.startCatalogCluster();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
index a8924dd..843df23 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
@@ -18,14 +18,13 @@
package org.apache.tajo.master;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.TestTajoIds;
import org.apache.tajo.master.ExecutionBlock.PartitionType;
import org.apache.tajo.master.querymaster.QueryUnit;
import org.apache.tajo.master.querymaster.Repartitioner;
import org.apache.tajo.util.TUtil;
-import org.apache.tajo.util.TajoIdUtils;
import org.jboss.netty.handler.codec.http.QueryStringDecoder;
import org.junit.Test;
@@ -37,10 +36,10 @@ import static junit.framework.Assert.assertEquals;
public class TestRepartitioner {
@Test
public void testCreateHashFetchURL() throws Exception {
- QueryId q1 = TestTajoIds.createQueryId(1315890136000l, 2, 1);
+ QueryId q1 = TestTajoIds.createQueryId(1315890136000l, 2);
String hostName = "tajo1";
int port = 1234;
- SubQueryId sid = TajoIdUtils.createSubQueryId(q1, 2);
+ ExecutionBlockId sid = new ExecutionBlockId(q1, 2);
int partitionId = 2;
List<QueryUnit.IntermediateEntry> intermediateEntries = TUtil.newList();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
index 05a269e..952cb0f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TaskRunnerTest.java
@@ -25,31 +25,27 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.tajo.QueryConf;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.TestTajoIds;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService;
import org.apache.tajo.rpc.ProtoAsyncRpcClient;
-import org.apache.tajo.util.TajoIdUtils;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TaskRunnerTest {
long ts1 = 1315890136000l;
- QueryId q1 = TestTajoIds.createQueryId(ts1, 2, 5);
- SubQueryId sq1 = TajoIdUtils.createSubQueryId(q1, 5);
+ QueryId q1 = TestTajoIds.createQueryId(ts1, 2);
+ ExecutionBlockId sq1 = QueryIdFactory.newExecutionBlockId(q1, 5);
//@Test
public void testInit() throws Exception {
ProtoAsyncRpcClient mockClient = mock(ProtoAsyncRpcClient.class);
mockClient.close();
- QueryMasterProtocolService.Interface mockMaster =
- mock(QueryMasterProtocolService.Interface.class);
+ TajoWorkerProtocolService.Interface mockMaster =
+ mock(TajoWorkerProtocolService.Interface.class);
ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
- q1.getApplicationId(), q1.getAttemptId());
+ BuilderUtils.newApplicationId(Integer.parseInt(q1.getId()), q1.getSeq()), 1);
ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 1);
NodeId nodeId = RecordFactoryProvider.getRecordFactory(null).
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 09ab483..cf1e9ae 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -23,7 +23,6 @@ import com.google.common.collect.Maps;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -73,7 +72,6 @@ public class TestRangeRetrieverHandler {
@Before
public void setUp() throws Exception {
- QueryIdFactory.reset();
util = new TajoTestingCluster();
conf = util.getConfiguration();
testDir = CommonTestingUtil.getTestDir("target/test-data/TestRangeRetrieverHandler");
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
index 6e6fdae..b70dda2 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
@@ -19,15 +19,15 @@
package org.apache.tajo.worker.dataserver;
import org.apache.hadoop.net.NetUtils;
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.util.CommonTestingUtil;
import org.apache.tajo.worker.InterDataRetriever;
import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
import org.apache.tajo.worker.dataserver.retriever.DirectoryRetriever;
+import org.junit.Before;
+import org.junit.Test;
import java.io.*;
import java.net.InetSocketAddress;
@@ -77,8 +77,7 @@ public class TestHttpDataServer {
@Test
public final void testInterDataRetriver() throws Exception {
- QueryIdFactory.reset();
- SubQueryId schid = QueryIdFactory.newSubQueryId(
+ ExecutionBlockId schid = QueryIdFactory.newExecutionBlockId(
QueryIdFactory.newQueryId());
QueryUnitId qid1 = QueryIdFactory.newQueryUnitId(schid);
QueryUnitId qid2 = QueryIdFactory.newQueryUnitId(schid);
@@ -119,8 +118,7 @@ public class TestHttpDataServer {
@Test(expected = FileNotFoundException.class)
public final void testNoSuchFile() throws Exception {
- QueryIdFactory.reset();
- SubQueryId schid = QueryIdFactory.newSubQueryId(
+ ExecutionBlockId schid = QueryIdFactory.newExecutionBlockId(
QueryIdFactory.newQueryId());
QueryUnitId qid1 = QueryIdFactory.newQueryUnitId(schid);
QueryUnitId qid2 = QueryIdFactory.newQueryUnitId(schid);
[6/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java
deleted file mode 100644
index e6d4c56..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskRunnerLauncherImpl.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
-import org.apache.tajo.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
-import org.apache.tajo.master.querymaster.QueryMaster;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
-import org.apache.tajo.worker.TaskRunner;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Vector;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-public class TaskRunnerLauncherImpl extends AbstractService implements TaskRunnerLauncher {
-
- /** Class Logger */
- private static final Log LOG = LogFactory.getLog(TaskRunnerLauncherImpl.class);
- private QueryContext context;
- private final String queryMasterHost;
- private final int queryMasterPort;
-
- // For ContainerLauncherSpec
- private ContainerLaunchContext commonContainerSpec = null;
-
- /** for launching TaskRunners in parallel */
- private final ExecutorService executorService;
-
- public TaskRunnerLauncherImpl(QueryContext context) {
- super(TaskRunnerLauncherImpl.class.getName());
- this.context = context;
- queryMasterHost = context.getQueryMasterServiceAddress().getHostName();
- queryMasterPort = context.getQueryMasterServiceAddress().getPort();
- executorService = Executors.newFixedThreadPool(
- context.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
- }
-
- public void start() {
- super.start();
- }
-
- public void stop() {
- executorService.shutdownNow();
- Map<ContainerId, ContainerProxy> containers = context.getContainers();
- for(ContainerProxy eachProxy: containers.values()) {
- try {
- eachProxy.kill();
- } catch (Exception e) {
- }
- }
- super.stop();
- }
-
- @Override
- public void handle(TaskRunnerGroupEvent event) {
- if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) {
- launchTaskRunners(event.subQueryId, event.getContainers());
- } else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
- killTaskRunners(event.getContainers());
- }
- }
-
- private void launchTaskRunners(SubQueryId subQueryId, Collection<Container> containers) {
- commonContainerSpec = ContainerProxy.createCommonContainerLaunchContext(getConfig(), subQueryId.toString(), false);
- for (Container container : containers) {
- final ContainerProxy proxy =
- new TaskRunnerContainerProxy(context, getConfig(), context.getYarnRPC(), container, subQueryId);
- executorService.submit(new LaunchRunner(container.getId(), proxy));
- }
- }
-
- private class LaunchRunner implements Runnable {
- private final ContainerProxy proxy;
- private final ContainerId id;
-
- public LaunchRunner(ContainerId id, ContainerProxy proxy) {
- this.proxy = proxy;
- this.id = id;
- }
- @Override
- public void run() {
- proxy.launch(commonContainerSpec);
- LOG.info("ContainerProxy started:" + id);
- }
- }
-
- private void killTaskRunners(Collection<Container> containers) {
- for (Container container : containers) {
- final ContainerProxy proxy = context.getContainer(container.getId());
- executorService.submit(new KillRunner(container.getId(), proxy));
- }
- }
-
- private class KillRunner implements Runnable {
- private final ContainerProxy proxy;
- private final ContainerId id;
- public KillRunner(ContainerId id, ContainerProxy proxy) {
- this.id = id;
- this.proxy = proxy;
- }
-
- @Override
- public void run() {
- proxy.kill();
- LOG.info("ContainerProxy killed:" + id);
- }
- }
-
- public class TaskRunnerContainerProxy extends ContainerProxy {
- private final SubQueryId subQueryId;
-
- public TaskRunnerContainerProxy(QueryMaster.QueryContext context, Configuration conf, YarnRPC yarnRPC,
- Container container, SubQueryId subQueryId) {
- super(context, conf, yarnRPC, container);
- this.subQueryId = subQueryId;
- }
-
- @Override
- protected void containerStarted() {
- context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
- }
-
- @Override
- protected String getId() {
- return subQueryId.toString();
- }
-
- @Override
- protected String getRunnerClass() {
- return TaskRunner.class.getCanonicalName();
- }
-
- @Override
- protected Vector<CharSequence> getTaskParams() {
- Vector<CharSequence> taskParams = new Vector<CharSequence>();
- taskParams.add(queryMasterHost); // queryMaster hostname
- taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
-
- return taskParams;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
index c9702b4..651f9c0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
@@ -23,28 +23,25 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.engine.planner.logical.ScanNode;
import org.apache.tajo.engine.query.QueryUnitRequestImpl;
-import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
import org.apache.tajo.master.event.TaskAttemptAssignedEvent;
import org.apache.tajo.master.event.TaskRequestEvent;
-import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
import org.apache.tajo.master.event.TaskScheduleEvent;
import org.apache.tajo.master.event.TaskSchedulerEvent;
import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
import org.apache.tajo.master.querymaster.QueryUnit;
import org.apache.tajo.storage.Fragment;
import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
import java.net.URI;
import java.util.*;
@@ -55,8 +52,8 @@ public class TaskSchedulerImpl extends AbstractService
implements TaskScheduler {
private static final Log LOG = LogFactory.getLog(TaskScheduleEvent.class);
- private final QueryContext context;
- private AsyncDispatcher dispatcher;
+ private final QueryMasterTask.QueryContext context;
+ private TajoAsyncDispatcher dispatcher;
private Thread eventHandlingThread;
private Thread schedulingThread;
@@ -72,21 +69,22 @@ public class TaskSchedulerImpl extends AbstractService
private int rackLocalAssigned = 0;
private int totalAssigned = 0;
- public TaskSchedulerImpl(QueryContext context) {
+ public TaskSchedulerImpl(QueryMasterTask.QueryContext context) {
super(TaskSchedulerImpl.class.getName());
this.context = context;
this.dispatcher = context.getDispatcher();
}
+ @Override
public void init(Configuration conf) {
scheduledRequests = new ScheduledRequests();
taskRequests = new TaskRequests();
- dispatcher.register(TaskRequestEventType.class, taskRequests);
super.init(conf);
}
+ @Override
public void start() {
LOG.info("Start TaskScheduler");
this.eventHandlingThread = new Thread() {
@@ -113,13 +111,14 @@ public class TaskSchedulerImpl extends AbstractService
while(!stopEventHandling && !Thread.currentThread().isInterrupted()) {
try {
- Thread.sleep(100);
+ Thread.sleep(1000);
} catch (InterruptedException e) {
break;
}
schedule();
}
+ //req.getCallback().run(stopTaskRunnerReq);
LOG.info("TaskScheduler schedulingThread stopped");
}
};
@@ -128,15 +127,15 @@ public class TaskSchedulerImpl extends AbstractService
super.start();
}
- private static final QueryUnitAttemptId NULL_ID;
- private static final QueryMasterProtocol.QueryUnitRequestProto stopTaskRunnerReq;
+ private static final QueryUnitAttemptId NULL_ATTEMPT_ID;
+ public static final TajoWorkerProtocol.QueryUnitRequestProto stopTaskRunnerReq;
static {
- SubQueryId nullSubQuery =
- QueryIdFactory.newSubQueryId(TajoIdUtils.NullQueryId);
- NULL_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
+ ExecutionBlockId nullSubQuery = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+ NULL_ATTEMPT_ID = QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(nullSubQuery, 0), 0);
- QueryMasterProtocol.QueryUnitRequestProto.Builder builder = QueryMasterProtocol.QueryUnitRequestProto.newBuilder();
- builder.setId(NULL_ID.getProto());
+ TajoWorkerProtocol.QueryUnitRequestProto.Builder builder =
+ TajoWorkerProtocol.QueryUnitRequestProto.newBuilder();
+ builder.setId(NULL_ATTEMPT_ID.getProto());
builder.setShouldDie(true);
builder.setOutputTable("");
builder.setSerializedData("");
@@ -144,7 +143,7 @@ public class TaskSchedulerImpl extends AbstractService
stopTaskRunnerReq = builder.build();
}
-
+ @Override
public void stop() {
stopEventHandling = true;
eventHandlingThread.interrupt();
@@ -205,12 +204,12 @@ public class TaskSchedulerImpl extends AbstractService
public void handle(TaskSchedulerEvent event) {
int qSize = eventQueue.size();
if (qSize != 0 && qSize % 1000 == 0) {
- LOG.info("Size of event-queue in RMContainerAllocator is " + qSize);
+ LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
}
int remCapacity = eventQueue.remainingCapacity();
if (remCapacity < 1000) {
LOG.warn("Very low remaining capacity in the event-queue "
- + "of RMContainerAllocator: " + remCapacity);
+ + "of YarnRMContainerAllocator: " + remCapacity);
}
try {
@@ -220,20 +219,29 @@ public class TaskSchedulerImpl extends AbstractService
}
}
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+ taskRequests.handle(event);
+ }
+
private class TaskRequests implements EventHandler<TaskRequestEvent> {
private final LinkedBlockingQueue<TaskRequestEvent> taskRequestQueue =
new LinkedBlockingQueue<TaskRequestEvent>();
@Override
public void handle(TaskRequestEvent event) {
+ LOG.info("====>TaskRequest:" + event.getContainerId() + "," + event.getExecutionBlockId());
+ if(stopEventHandling) {
+ event.getCallback().run(stopTaskRunnerReq);
+ return;
+ }
int qSize = taskRequestQueue.size();
if (qSize != 0 && qSize % 1000 == 0) {
- LOG.info("Size of event-queue in RMContainerAllocator is " + qSize);
+ LOG.info("Size of event-queue in YarnRMContainerAllocator is " + qSize);
}
int remCapacity = taskRequestQueue.remainingCapacity();
if (remCapacity < 1000) {
LOG.warn("Very low remaining capacity in the event-queue "
- + "of RMContainerAllocator: " + remCapacity);
+ + "of YarnRMContainerAllocator: " + remCapacity);
}
taskRequestQueue.add(event);
@@ -380,15 +388,16 @@ public class TaskSchedulerImpl extends AbstractService
return nonLeafTasks.size();
}
- public Set<QueryUnitAttemptId> AssignedRequest = new HashSet<QueryUnitAttemptId>();
+ public Set<QueryUnitAttemptId> assignedRequest = new HashSet<QueryUnitAttemptId>();
+
public void assignToLeafTasks(List<TaskRequestEvent> taskRequests) {
Iterator<TaskRequestEvent> it = taskRequests.iterator();
- LOG.info("Got task requests " + taskRequests.size());
TaskRequestEvent taskRequest;
while (it.hasNext() && leafTasks.size() > 0) {
taskRequest = it.next();
- ContainerProxy container = context.getContainer(taskRequest.getContainerId());
+ LOG.info("====> assignToLeafTasks: " + taskRequest.getExecutionBlockId());
+ ContainerProxy container = context.getResourceAllocator().getContainer(taskRequest.getContainerId());
String host = container.getTaskHostName();
QueryUnitAttemptId attemptId = null;
@@ -443,7 +452,7 @@ public class TaskSchedulerImpl extends AbstractService
if (attemptId != null) {
QueryUnit task = context.getQuery()
- .getSubQuery(attemptId.getSubQueryId()).getQueryUnit(attemptId.getQueryUnitId());
+ .getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId()).getQueryUnit(attemptId.getQueryUnitId());
QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
attemptId,
new ArrayList<Fragment>(task.getAllFragments()),
@@ -457,7 +466,7 @@ public class TaskSchedulerImpl extends AbstractService
context.getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
taskRequest.getContainerId(),
host, container.getTaskPort()));
- AssignedRequest.add(attemptId);
+ assignedRequest.add(attemptId);
totalAssigned++;
taskRequest.getCallback().run(taskAssign.getProto());
@@ -476,6 +485,7 @@ public class TaskSchedulerImpl extends AbstractService
TaskRequestEvent taskRequest;
while (it.hasNext()) {
taskRequest = it.next();
+ LOG.info("====> assignToNonLeafTasks: " + taskRequest.getExecutionBlockId());
QueryUnitAttemptId attemptId;
// random allocation
@@ -485,7 +495,8 @@ public class TaskSchedulerImpl extends AbstractService
LOG.debug("Assigned based on * match");
QueryUnit task;
- task = context.getSubQuery(attemptId.getSubQueryId()).getQueryUnit(attemptId.getQueryUnitId());
+ task = context.getSubQuery(
+ attemptId.getQueryUnitId().getExecutionBlockId()).getQueryUnit(attemptId.getQueryUnitId());
QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
attemptId,
Lists.newArrayList(task.getAllFragments()),
@@ -504,7 +515,7 @@ public class TaskSchedulerImpl extends AbstractService
}
}
- ContainerProxy container = context.getContainer(
+ ContainerProxy container = context.getResourceAllocator().getContainer(
taskRequest.getContainerId());
context.getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
taskRequest.getContainerId(), container.getTaskHostName(), container.getTaskPort()));
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
new file mode 100644
index 0000000..1e650e1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnContainerProxy.java
@@ -0,0 +1,446 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.ProtoUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.event.QueryEvent;
+import org.apache.tajo.master.event.QueryEventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.pullserver.PullServerAuxService;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.util.*;
+
+public class YarnContainerProxy extends ContainerProxy {
+ private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+
+ protected final YarnRPC yarnRPC;
+ final protected String containerMgrAddress;
+ protected ContainerToken containerToken;
+
+ public YarnContainerProxy(QueryMasterTask.QueryContext context, Configuration conf, YarnRPC yarnRPC,
+ Container container, ExecutionBlockId executionBlockId) {
+ super(context, conf, executionBlockId, container);
+ this.yarnRPC = yarnRPC;
+
+ NodeId nodeId = container.getNodeId();
+ this.containerMgrAddress = nodeId.getHost() + ":" + nodeId.getPort();
+ this.containerToken = container.getContainerToken();
+ }
+
+ protected ContainerManager getCMProxy(ContainerId containerID,
+ final String containerManagerBindAddr,
+ ContainerToken containerToken)
+ throws IOException {
+ String [] hosts = containerManagerBindAddr.split(":");
+ final InetSocketAddress cmAddr =
+ new InetSocketAddress(hosts[0], Integer.parseInt(hosts[1]));
+ UserGroupInformation user = UserGroupInformation.getCurrentUser();
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ Token<ContainerTokenIdentifier> token =
+ ProtoUtils.convertFromProtoFormat(containerToken, cmAddr);
+ // the user in createRemoteUser in this context has to be ContainerID
+ user = UserGroupInformation.createRemoteUser(containerID.toString());
+ user.addToken(token);
+ }
+
+ ContainerManager proxy = user.doAs(new PrivilegedAction<ContainerManager>() {
+ @Override
+ public ContainerManager run() {
+ return (ContainerManager) yarnRPC.getProxy(ContainerManager.class,
+ cmAddr, conf);
+ }
+ });
+
+ return proxy;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public synchronized void launch(ContainerLaunchContext commonContainerLaunchContext) {
+ LOG.info("Launching Container with Id: " + containerID);
+ if(this.state == ContainerState.KILLED_BEFORE_LAUNCH) {
+ state = ContainerState.DONE;
+ LOG.error("Container (" + containerID + " was killed before it was launched");
+ return;
+ }
+
+ ContainerManager proxy = null;
+ try {
+
+ proxy = getCMProxy(containerID, containerMgrAddress,
+ containerToken);
+
+ // Construct the actual Container
+ ContainerLaunchContext containerLaunchContext = createContainerLaunchContext(commonContainerLaunchContext);
+
+ // Now launch the actual container
+ StartContainerRequest startRequest = Records
+ .newRecord(StartContainerRequest.class);
+ startRequest.setContainerLaunchContext(containerLaunchContext);
+ StartContainerResponse response = proxy.startContainer(startRequest);
+
+ ByteBuffer portInfo = response
+ .getServiceResponse(PullServerAuxService.PULLSERVER_SERVICEID);
+
+ if(portInfo != null) {
+ port = PullServerAuxService.deserializeMetaData(portInfo);
+ }
+
+ LOG.info("PullServer port returned by ContainerManager for "
+ + containerID + " : " + port);
+
+ if(port < 0) {
+ this.state = ContainerState.FAILED;
+ throw new IllegalStateException("Invalid shuffle port number "
+ + port + " returned for " + containerID);
+ }
+
+ context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+
+ this.state = ContainerState.RUNNING;
+ this.hostName = containerMgrAddress.split(":")[0];
+ context.getResourceAllocator().addContainer(containerID, this);
+ } catch (Throwable t) {
+ String message = "Container launch failed for " + containerID + " : "
+ + StringUtils.stringifyException(t);
+ this.state = ContainerState.FAILED;
+ LOG.error(message);
+ } finally {
+ if (proxy != null) {
+ yarnRPC.stopProxy(proxy, conf);
+ }
+ }
+ }
+
+
+ public ContainerLaunchContext createContainerLaunchContext(ContainerLaunchContext commonContainerLaunchContext) {
+ // Setup environment by cloning from common env.
+ Map<String, String> env = commonContainerLaunchContext.getEnvironment();
+ Map<String, String> myEnv = new HashMap<String, String>(env.size());
+ myEnv.putAll(env);
+
+ // Duplicate the ByteBuffers for access by multiple containers.
+ Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
+ for (Map.Entry<String, ByteBuffer> entry : commonContainerLaunchContext.getServiceData().entrySet()) {
+ myServiceData.put(entry.getKey(), entry.getValue().duplicate());
+ }
+
+ ////////////////////////////////////////////////////////////////////////////
+ // Set the local resources
+ ////////////////////////////////////////////////////////////////////////////
+ // Set the necessary command to execute the application master
+ Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+ // Set java executable command
+ //LOG.info("Setting up app master command");
+ vargs.add("${JAVA_HOME}" + "/bin/java");
+ // Set Xmx based on am memory size
+ vargs.add("-Xmx2000m");
+ // Set Remote Debugging
+ //if (!context.getQuery().getSubQuery(event.getExecutionBlockId()).isLeafQuery()) {
+ //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
+ //}
+ // Set class name
+ //vargs.add(getRunnerClass());
+ vargs.add(TajoWorker.class.getCanonicalName());
+ vargs.add("tr"); //workerMode
+ vargs.add(getId()); // subqueryId
+ vargs.add(containerMgrAddress); // nodeId
+ vargs.add(containerID.toString()); // containerId
+ Vector<CharSequence> taskParams = getTaskParams();
+ if(taskParams != null) {
+ vargs.addAll(taskParams);
+ }
+
+ vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+ vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+ // Get final commmand
+ StringBuilder command = new StringBuilder();
+ for (CharSequence str : vargs) {
+ command.append(str).append(" ");
+ }
+
+ LOG.info("Completed setting up TaskRunner command " + command.toString());
+ List<String> commands = new ArrayList<String>();
+ commands.add(command.toString());
+
+ return BuilderUtils.newContainerLaunchContext(containerID, commonContainerLaunchContext.getUser(),
+ container.getResource(), commonContainerLaunchContext.getLocalResources(), myEnv, commands,
+ myServiceData, null, new HashMap<ApplicationAccessType, String>());
+ }
+
+ public static ContainerLaunchContext createCommonContainerLaunchContext(Configuration config,
+ String queryId, boolean isMaster) {
+ TajoConf conf = (TajoConf)config;
+
+ ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class);
+ try {
+ ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ ////////////////////////////////////////////////////////////////////////////
+ // Set the env variables to be setup
+ ////////////////////////////////////////////////////////////////////////////
+ LOG.info("Set the environment for the application master");
+
+ Map<String, String> environment = new HashMap<String, String>();
+ //String initialClassPath = getInitialClasspath(conf);
+ environment.put(ApplicationConstants.Environment.SHELL.name(), "/bin/bash");
+ if(System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()) != null) {
+ environment.put(ApplicationConstants.Environment.JAVA_HOME.name(), System.getenv(ApplicationConstants.Environment.JAVA_HOME.name()));
+ }
+
+ // TODO - to be improved with org.apache.tajo.sh shell script
+ Properties prop = System.getProperties();
+
+ if (prop.getProperty("tajo.test", "FALSE").equalsIgnoreCase("TRUE") ||
+ (System.getenv("tajo.test") != null && System.getenv("tajo.test").equalsIgnoreCase("TRUE"))) {
+ LOG.info("=========> tajo.test is TRUE");
+ environment.put(ApplicationConstants.Environment.CLASSPATH.name(), prop.getProperty(
+ "java.class.path", null));
+ environment.put("tajo.test", "TRUE");
+ } else {
+ // Add AppMaster.jar location to classpath
+ // At some point we should not be required to add
+ // the hadoop specific classpaths to the env.
+ // It should be provided out of the box.
+ // For now setting all required classpaths including
+ // the classpath to "." for the application jar
+ StringBuilder classPathEnv = new StringBuilder("./");
+ //for (String c : conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+ for (String c : YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH) {
+ classPathEnv.append(':');
+ classPathEnv.append(c.trim());
+ }
+
+ classPathEnv.append(":" + System.getenv("TAJO_BASE_CLASSPATH"));
+ classPathEnv.append(":./log4j.properties:./*");
+ if(System.getenv("HADOOP_HOME") != null) {
+ environment.put("HADOOP_HOME", System.getenv("HADOOP_HOME"));
+ environment.put(
+ ApplicationConstants.Environment.HADOOP_COMMON_HOME.name(),
+ System.getenv("HADOOP_HOME"));
+ environment.put(
+ ApplicationConstants.Environment.HADOOP_HDFS_HOME.name(),
+ System.getenv("HADOOP_HOME"));
+ environment.put(
+ ApplicationConstants.Environment.HADOOP_YARN_HOME.name(),
+ System.getenv("HADOOP_HOME"));
+ }
+
+ if(System.getenv("TAJO_BASE_CLASSPATH") != null) {
+ environment.put("TAJO_BASE_CLASSPATH", System.getenv("TAJO_BASE_CLASSPATH"));
+ }
+ environment.put(ApplicationConstants.Environment.CLASSPATH.name(), classPathEnv.toString());
+ }
+
+ ctx.setEnvironment(environment);
+
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("=================================================");
+ for(Map.Entry<String, String> entry: environment.entrySet()) {
+ LOG.debug(entry.getKey() + "=" + entry.getValue());
+ }
+ LOG.debug("=================================================");
+ }
+ ////////////////////////////////////////////////////////////////////////////
+ // Set the local resources
+ ////////////////////////////////////////////////////////////////////////////
+ Map<String, LocalResource> localResources = new HashMap<String, LocalResource>();
+ FileSystem fs = null;
+
+ LOG.info("defaultFS: " + conf.get("fs.default.name"));
+ LOG.info("defaultFS: " + conf.get("fs.defaultFS"));
+ try {
+ fs = FileSystem.get(conf);
+ } catch (IOException e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ FileContext fsCtx = null;
+ try {
+ fsCtx = FileContext.getFileContext(conf);
+ } catch (UnsupportedFileSystemException e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ LOG.info("Writing a QueryConf to HDFS and add to local environment");
+ //Path queryConfPath = new Path(fs.getHomeDirectory(), QueryConf.FILENAME);
+ try {
+ //writeConf(conf, queryConfPath);
+ // TODO move to tajo temp
+ Path warehousePath = new Path(conf.getVar(TajoConf.ConfVars.ROOT_DIR), TajoConstants.WAREHOUSE_DIR);
+ Path queryConfPath = new Path(warehousePath, queryId);
+ if(isMaster) {
+ queryConfPath = new Path(queryConfPath, QueryConf.QUERY_MASTER_FILENAME);
+ } else {
+ queryConfPath = new Path(queryConfPath, QueryConf.FILENAME);
+ }
+
+ if(!fs.exists(queryConfPath)){
+ writeConf(conf, queryConfPath);
+ } else {
+ LOG.warn("QueryConf already exist. path: " + queryConfPath.toString());
+ }
+
+ LocalResource queryConfSrc = createApplicationResource(fsCtx,
+ queryConfPath, LocalResourceType.FILE);
+// localResources.put(QueryConf.FILENAME, queryConfSrc);
+ localResources.put(queryConfPath.getName(), queryConfSrc);
+
+ ctx.setLocalResources(localResources);
+ } catch (IOException e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ // TODO - move to sub-class
+ // Add shuffle token
+ Map<String, ByteBuffer> serviceData = new HashMap<String, ByteBuffer>();
+ try {
+ //LOG.info("Putting shuffle token in serviceData");
+ serviceData.put(PullServerAuxService.PULLSERVER_SERVICEID,
+ PullServerAuxService.serializeMetaData(0));
+ } catch (IOException ioe) {
+ LOG.error(ioe);
+ }
+ ctx.setServiceData(serviceData);
+
+ return ctx;
+ }
+
+ private static LocalResource createApplicationResource(FileContext fs,
+ Path p, LocalResourceType type)
+ throws IOException {
+ LocalResource rsrc = recordFactory.newRecordInstance(LocalResource.class);
+ FileStatus rsrcStat = fs.getFileStatus(p);
+ rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs
+ .getDefaultFileSystem().resolvePath(rsrcStat.getPath())));
+ rsrc.setSize(rsrcStat.getLen());
+ rsrc.setTimestamp(rsrcStat.getModificationTime());
+ rsrc.setType(type);
+ rsrc.setVisibility(LocalResourceVisibility.APPLICATION);
+ return rsrc;
+ }
+
+ private static void writeConf(Configuration conf, Path queryConfFile)
+ throws IOException {
+ // Write job file to Tajo's fs
+ FileSystem fs = queryConfFile.getFileSystem(conf);
+ FSDataOutputStream out =
+ FileSystem.create(fs, queryConfFile,
+ new FsPermission(QUERYCONF_FILE_PERMISSION));
+ try {
+ conf.writeXml(out);
+ } finally {
+ out.close();
+ }
+ }
+
+ @Override
+ public synchronized void stopContainer() {
+
+ if(isCompletelyDone()) {
+ return;
+ }
+ if(this.state == ContainerState.PREP) {
+ this.state = ContainerState.KILLED_BEFORE_LAUNCH;
+ } else {
+ LOG.info("KILLING " + containerID);
+
+ ContainerManager proxy = null;
+ try {
+ proxy = getCMProxy(this.containerID, this.containerMgrAddress,
+ this.containerToken);
+
+ // kill the remote container if already launched
+ StopContainerRequest stopRequest = Records
+ .newRecord(StopContainerRequest.class);
+ stopRequest.setContainerId(this.containerID);
+ proxy.stopContainer(stopRequest);
+ // If stopContainer returns without an error, assuming the stop made
+ // it over to the NodeManager.
+// context.getEventHandler().handle(
+// new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
+ context.getResourceAllocator().removeContainer(containerID);
+ } catch (Throwable t) {
+
+ // ignore the cleanup failure
+ String message = "cleanup failed for container "
+ + this.containerID + " : "
+ + StringUtils.stringifyException(t);
+// context.getEventHandler().handle(
+// new AMContainerEventStopFailed(containerID, message));
+ LOG.warn(message);
+ this.state = ContainerState.DONE;
+ return;
+ } finally {
+ if (proxy != null) {
+ yarnRPC.stopProxy(proxy, conf);
+ }
+ }
+ this.state = ContainerState.DONE;
+ }
+ }
+
+ protected Vector<CharSequence> getTaskParams() {
+ String queryMasterHost = context.getQueryMasterContext().getWorkerContext()
+ .getTajoWorkerManagerService().getBindAddr().getHostName();
+ int queryMasterPort = context.getQueryMasterContext().getWorkerContext()
+ .getTajoWorkerManagerService().getBindAddr().getPort();
+
+ Vector<CharSequence> taskParams = new Vector<CharSequence>();
+ taskParams.add(queryMasterHost); // queryMaster hostname
+ taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
+ taskParams.add(context.getOutputPath().toString());
+ return taskParams;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
new file mode 100644
index 0000000..5ac4fb5
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/YarnTaskRunnerLauncherImpl.java
@@ -0,0 +1,208 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class YarnTaskRunnerLauncherImpl extends AbstractService implements TaskRunnerLauncher {
+
+ /** Class Logger */
+ private static final Log LOG = LogFactory.getLog(YarnTaskRunnerLauncherImpl.class);
+ //private final YarnRPC yarnRPC;
+ private final static RecordFactory recordFactory =
+ RecordFactoryProvider.getRecordFactory(null);
+ private QueryMasterTask.QueryContext context;
+
+ // For ContainerLauncherSpec
+ private static AtomicBoolean initialClasspathFlag = new AtomicBoolean();
+ private static String initialClasspath = null;
+ private static final Object classpathLock = new Object();
+ private ContainerLaunchContext commonContainerSpec = null;
+
+ final public static FsPermission QUERYCONF_FILE_PERMISSION =
+ FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+ /** for launching TaskRunners in parallel */
+ private final ExecutorService executorService;
+
+ private YarnRPC yarnRPC;
+
+ public YarnTaskRunnerLauncherImpl(QueryMasterTask.QueryContext context, YarnRPC yarnRPC) {
+ super(YarnTaskRunnerLauncherImpl.class.getName());
+ this.context = context;
+ this.yarnRPC = yarnRPC;
+ executorService = Executors.newFixedThreadPool(
+ context.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+ }
+
+ public void start() {
+ super.start();
+ }
+
+ public void stop() {
+ executorService.shutdownNow();
+
+ while(!executorService.isTerminated()) {
+ LOG.info("====>executorService.isTerminated:" + executorService.isTerminated() + "," + executorService.isShutdown());
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ }
+ }
+ Map<ContainerId, ContainerProxy> containers = context.getResourceAllocator().getContainers();
+ for(ContainerProxy eachProxy: containers.values()) {
+ try {
+ eachProxy.stopContainer();
+ } catch (Exception e) {
+ }
+ }
+ super.stop();
+ }
+
+ @Override
+ public void handle(TaskRunnerGroupEvent event) {
+ if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) {
+ launchTaskRunners(event.executionBlockId, event.getContainers());
+ } else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
+ stopTaskRunners(event.getContainers());
+ }
+ }
+
+ private void launchTaskRunners(ExecutionBlockId executionBlockId, Collection<Container> containers) {
+ commonContainerSpec = YarnContainerProxy.createCommonContainerLaunchContext(getConfig(),
+ executionBlockId.getQueryId().toString(), false);
+ for (Container container : containers) {
+ final ContainerProxy proxy = new YarnContainerProxy(context, getConfig(),
+ yarnRPC, container, executionBlockId);
+ executorService.submit(new LaunchRunner(container.getId(), proxy));
+ }
+ }
+
+ protected class LaunchRunner implements Runnable {
+ private final ContainerProxy proxy;
+ private final ContainerId id;
+ public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+ this.proxy = proxy;
+ this.id = id;
+ }
+ @Override
+ public void run() {
+ proxy.launch(commonContainerSpec);
+ LOG.info("ContainerProxy started:" + id);
+ }
+ }
+
+ private void stopTaskRunners(Collection<Container> containers) {
+ for (Container container : containers) {
+ final ContainerProxy proxy = context.getResourceAllocator().getContainer(container.getId());
+ executorService.submit(new StopContainerRunner(container.getId(), proxy));
+ }
+ }
+
+ private class StopContainerRunner implements Runnable {
+ private final ContainerProxy proxy;
+ private final ContainerId id;
+ public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+ this.id = id;
+ this.proxy = proxy;
+ }
+
+ @Override
+ public void run() {
+ proxy.stopContainer();
+ LOG.info("ContainerProxy stopped:" + id);
+ }
+ }
+
+
+ /**
+ * Lock this on initialClasspath so that there is only one fork in the AM for
+ * getting the initial class-path. TODO: We already construct
+ * a parent CLC and use it for all the containers, so this should go away
+ * once the mr-generated-classpath stuff is gone.
+ */
+ private static String getInitialClasspath(Configuration conf) {
+ synchronized (classpathLock) {
+ if (initialClasspathFlag.get()) {
+ return initialClasspath;
+ }
+ Map<String, String> env = new HashMap<String, String>();
+
+ initialClasspath = env.get(Environment.CLASSPATH.name());
+ initialClasspathFlag.set(true);
+ return initialClasspath;
+ }
+ }
+
+// public class TaskRunnerContainerProxy extends ContainerProxy {
+// private final ExecutionBlockId executionBlockId;
+//
+// public TaskRunnerContainerProxy(QueryMasterTask.QueryContext context, Configuration conf, YarnRPC yarnRPC,
+// Container container, ExecutionBlockId executionBlockId) {
+// super(context, conf, yarnRPC, container);
+// this.executionBlockId = executionBlockId;
+// }
+//
+// @Override
+// protected void containerStarted() {
+// context.getEventHandler().handle(new QueryEvent(context.getQueryId(), QueryEventType.INIT_COMPLETED));
+// }
+//
+// @Override
+// protected String getId() {
+// return executionBlockId.toString();
+// }
+//
+// @Override
+// protected String getRunnerClass() {
+// return TaskRunner.class.getCanonicalName();
+// }
+//
+// @Override
+// protected Vector<CharSequence> getTaskParams() {
+// Vector<CharSequence> taskParams = new Vector<CharSequence>();
+// taskParams.add(queryMasterHost); // queryMaster hostname
+// taskParams.add(String.valueOf(queryMasterPort)); // queryMaster port
+//
+// return taskParams;
+// }
+// }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
index 6704aa4..ee594a3 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/ContainerAllocationEvent.java
@@ -21,11 +21,11 @@ package org.apache.tajo.master.event;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEventType> {
- private final SubQueryId subQueryId;
+ private final ExecutionBlockId executionBlockId;
private final Priority priority;
private final Resource resource;
private final boolean isLeafQuery;
@@ -33,13 +33,13 @@ public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEv
private final float progress;
public ContainerAllocationEvent(ContainerAllocatorEventType eventType,
- SubQueryId subQueryId,
- Priority priority,
- Resource resource,
- int requiredNum,
- boolean isLeafQuery, float progress) {
+ ExecutionBlockId executionBlockId,
+ Priority priority,
+ Resource resource,
+ int requiredNum,
+ boolean isLeafQuery, float progress) {
super(eventType);
- this.subQueryId = subQueryId;
+ this.executionBlockId = executionBlockId;
this.priority = priority;
this.resource = resource;
this.requiredNum = requiredNum;
@@ -47,8 +47,8 @@ public class ContainerAllocationEvent extends AbstractEvent<ContainerAllocatorEv
this.progress = progress;
}
- public SubQueryId getSubQueryId() {
- return subQueryId;
+ public ExecutionBlockId getExecutionBlockId() {
+ return executionBlockId;
}
public Priority getPriority() {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
index 44abf30..c34b174 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/GrouppedContainerAllocatorEvent.java
@@ -20,7 +20,7 @@ package org.apache.tajo.master.event;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import java.util.Map;
@@ -29,12 +29,12 @@ public class GrouppedContainerAllocatorEvent
private final Map<String, Integer> requestMap;
public GrouppedContainerAllocatorEvent(ContainerAllocatorEventType eventType,
- SubQueryId subQueryId,
+ ExecutionBlockId executionBlockId,
Priority priority,
Resource resource,
Map<String, Integer> requestMap,
boolean isLeafQuery, float progress) {
- super(eventType, subQueryId, priority,
+ super(eventType, executionBlockId, priority,
resource, requestMap.size(), isLeafQuery, progress);
this.requestMap = requestMap;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
new file mode 100644
index 0000000..6ae8ff7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.event;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.QueryId;
+
+public class QueryStartEvent extends AbstractEvent {
+ public enum EventType {
+ QUERY_START
+ }
+
+ private QueryId queryId;
+ private String logicalPlanJson;
+
+ public QueryStartEvent(QueryId queryId, String logicalPlanJson) {
+ super(EventType.QUERY_START);
+ this.queryId = queryId;
+ this.logicalPlanJson = logicalPlanJson;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public String getLogicalPlanJson() {
+ return logicalPlanJson;
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getName() + "," + getType() + "," + queryId;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
index 10b67fe..ae36a69 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QuerySubQueryEvent.java
@@ -18,18 +18,18 @@
package org.apache.tajo.master.event;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
public class QuerySubQueryEvent extends QueryEvent {
- private SubQueryId subQueryId;
+ private ExecutionBlockId executionBlockId;
- public QuerySubQueryEvent(final SubQueryId id,
+ public QuerySubQueryEvent(final ExecutionBlockId id,
final QueryEventType queryEvent) {
super(id.getQueryId(), queryEvent);
- this.subQueryId = id;
+ this.executionBlockId = id;
}
- public SubQueryId getSubQueryId() {
- return this.subQueryId;
+ public ExecutionBlockId getExecutionBlockId() {
+ return this.executionBlockId;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
index 26c7231..7e07525 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryCompletedEvent.java
@@ -18,22 +18,22 @@
package org.apache.tajo.master.event;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.master.querymaster.SubQueryState;
public class SubQueryCompletedEvent extends QueryEvent {
- private final SubQueryId subQueryId;
+ private final ExecutionBlockId executionBlockId;
private final SubQueryState finalState;
- public SubQueryCompletedEvent(final SubQueryId subQueryId,
+ public SubQueryCompletedEvent(final ExecutionBlockId executionBlockId,
SubQueryState finalState) {
- super(subQueryId.getQueryId(), QueryEventType.SUBQUERY_COMPLETED);
- this.subQueryId = subQueryId;
+ super(executionBlockId.getQueryId(), QueryEventType.SUBQUERY_COMPLETED);
+ this.executionBlockId = executionBlockId;
this.finalState = finalState;
}
- public SubQueryId getSubQueryId() {
- return subQueryId;
+ public ExecutionBlockId getExecutionBlockId() {
+ return executionBlockId;
}
public SubQueryState getFinalState() {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
index 5c0ef9a..a8f4800 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryContainerAllocationEvent.java
@@ -19,14 +19,14 @@
package org.apache.tajo.master.event;
import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import java.util.List;
public class SubQueryContainerAllocationEvent extends SubQueryEvent {
private List<Container> allocatedContainer;
- public SubQueryContainerAllocationEvent(final SubQueryId id,
+ public SubQueryContainerAllocationEvent(final ExecutionBlockId id,
List<Container> allocatedContainer) {
super(id, SubQueryEventType.SQ_CONTAINER_ALLOCATED);
this.allocatedContainer = allocatedContainer;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
index 11470ed..2b3d598 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryEvent.java
@@ -19,17 +19,17 @@
package org.apache.tajo.master.event;
import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
public class SubQueryEvent extends AbstractEvent<SubQueryEventType> {
- private final SubQueryId id;
+ private final ExecutionBlockId id;
- public SubQueryEvent(SubQueryId id, SubQueryEventType subQueryEventType) {
+ public SubQueryEvent(ExecutionBlockId id, SubQueryEventType subQueryEventType) {
super(subQueryEventType);
this.id = id;
}
- public SubQueryId getSubQueryId() {
+ public ExecutionBlockId getSubQueryId() {
return id;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
index d85d4f2..e02196a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQuerySucceeEvent.java
@@ -18,14 +18,14 @@
package org.apache.tajo.master.event;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.catalog.TableMeta;
import org.apache.tajo.master.querymaster.SubQueryState;
public class SubQuerySucceeEvent extends SubQueryCompletedEvent {
private final TableMeta tableMeta;
- public SubQuerySucceeEvent(final SubQueryId id, TableMeta tableMeta) {
+ public SubQuerySucceeEvent(final ExecutionBlockId id, TableMeta tableMeta) {
super(id, SubQueryState.SUCCEEDED);
this.tableMeta = tableMeta;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
index 0315236..0217f20 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/SubQueryTaskEvent.java
@@ -27,7 +27,7 @@ public class SubQueryTaskEvent extends SubQueryEvent {
private QueryUnitId taskId;
public SubQueryTaskEvent(QueryUnitId taskId,
SubQueryEventType subQueryEventType) {
- super(taskId.getSubQueryId(), subQueryEventType);
+ super(taskId.getExecutionBlockId(), subQueryEventType);
this.taskId = taskId;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
index bc84011..d980e05 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskAttemptStatusUpdateEvent.java
@@ -19,7 +19,7 @@
package org.apache.tajo.master.event;
import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskStatusProto;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskStatusProto;
public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent {
private final TaskStatusProto status;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
index e3a4b5f..3ee389a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskCompletionEvent.java
@@ -19,7 +19,7 @@
package org.apache.tajo.master.event;
import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskCompletionReport;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
public class TaskCompletionEvent extends TaskAttemptEvent {
private TaskCompletionReport report;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
index 06fb392..d70de8a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskFatalErrorEvent.java
@@ -19,7 +19,7 @@
package org.apache.tajo.master.event;
import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskFatalErrorReport;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskFatalErrorReport;
public class TaskFatalErrorEvent extends TaskAttemptEvent {
private TaskFatalErrorReport report;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
index 166e103..9be7cab 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskRequestEvent.java
@@ -21,7 +21,8 @@ package org.apache.tajo.master.event;
import com.google.protobuf.RpcCallback;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryUnitRequestProto;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
import org.apache.tajo.master.event.TaskRequestEvent.TaskRequestEventType;
public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
@@ -31,12 +32,16 @@ public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
}
private final ContainerId workerId;
+ private final ExecutionBlockId executionBlockId;
+
private final RpcCallback<QueryUnitRequestProto> callback;
public TaskRequestEvent(ContainerId workerId,
+ ExecutionBlockId executionBlockId,
RpcCallback<QueryUnitRequestProto> callback) {
super(TaskRequestEventType.TASK_REQ);
this.workerId = workerId;
+ this.executionBlockId = executionBlockId;
this.callback = callback;
}
@@ -44,6 +49,10 @@ public class TaskRequestEvent extends AbstractEvent<TaskRequestEventType> {
return this.workerId;
}
+ public ExecutionBlockId getExecutionBlockId() {
+ return executionBlockId;
+ }
+
public RpcCallback<QueryUnitRequestProto> getCallback() {
return this.callback;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
index 1f87356..f460203 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskScheduleEvent.java
@@ -34,7 +34,7 @@ public class TaskScheduleEvent extends TaskSchedulerEvent {
final EventType eventType, boolean isLeafQuery,
final List<QueryUnit.DataLocation> dataLocations,
final String[] racks) {
- super(eventType, attemptId.getSubQueryId());
+ super(eventType, attemptId.getQueryUnitId().getExecutionBlockId());
this.attemptId = attemptId;
this.isLeafQuery = isLeafQuery;
this.dataLocations = dataLocations;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
index d73bb87..71d8587 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/TaskSchedulerEvent.java
@@ -19,7 +19,7 @@
package org.apache.tajo.master.event;
import org.apache.hadoop.yarn.event.AbstractEvent;
-import org.apache.tajo.SubQueryId;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
public class TaskSchedulerEvent extends AbstractEvent<EventType> {
@@ -28,14 +28,14 @@ public class TaskSchedulerEvent extends AbstractEvent<EventType> {
T_SUBQUERY_COMPLETED
}
- private final SubQueryId subQueryId;
+ private final ExecutionBlockId executionBlockId;
- public TaskSchedulerEvent(EventType eventType, SubQueryId subQueryId) {
+ public TaskSchedulerEvent(EventType eventType, ExecutionBlockId queryBlockId) {
super(eventType);
- this.subQueryId = subQueryId;
+ this.executionBlockId = queryBlockId;
}
- public SubQueryId getSubQueryId() {
- return this.subQueryId;
+ public ExecutionBlockId getExecutionBlockId() {
+ return this.executionBlockId;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 3179abf..99b7c62 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -25,9 +25,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryConf;
import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.TajoProtos.QueryState;
import org.apache.tajo.catalog.TableDesc;
import org.apache.tajo.catalog.TableDescImpl;
@@ -35,7 +35,6 @@ import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.master.ExecutionBlock;
import org.apache.tajo.master.ExecutionBlockCursor;
import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
import org.apache.tajo.storage.StorageManager;
import java.io.IOException;
@@ -50,16 +49,15 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
public class Query implements EventHandler<QueryEvent> {
private static final Log LOG = LogFactory.getLog(Query.class);
-
// Facilities for Query
private final QueryConf conf;
private final Clock clock;
private String queryStr;
- private Map<SubQueryId, SubQuery> subqueries;
+ private Map<ExecutionBlockId, SubQuery> subqueries;
private final EventHandler eventHandler;
private final MasterPlan plan;
private final StorageManager sm;
- private QueryContext context;
+ private QueryMasterTask.QueryContext context;
private ExecutionBlockCursor cursor;
// Query Status
@@ -106,22 +104,21 @@ public class Query implements EventHandler<QueryEvent> {
.installTopology();
- public Query(final QueryContext context, final QueryId id, Clock clock,
+ public Query(final QueryMasterTask.QueryContext context, final QueryId id,
final long appSubmitTime,
final String queryStr,
final EventHandler eventHandler,
- final MasterPlan plan,
- final StorageManager sm) {
+ final MasterPlan plan) {
this.context = context;
this.conf = context.getConf();
this.id = id;
- this.clock = clock;
+ this.clock = context.getClock();
this.appSubmitTime = appSubmitTime;
this.queryStr = queryStr;
subqueries = Maps.newHashMap();
this.eventHandler = eventHandler;
this.plan = plan;
- this.sm = sm;
+ this.sm = context.getStorageManager();
cursor = new ExecutionBlockCursor(plan);
ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
@@ -131,23 +128,19 @@ public class Query implements EventHandler<QueryEvent> {
stateMachine = stateMachineFactory.make(this);
}
- public boolean isCreateTableStmt() {
- return context.isCreateTableQuery();
- }
-
-// protected FileSystem getFileSystem(Configuration conf) throws IOException {
-// return FileSystem.get(conf);
-// }
-
public float getProgress() {
QueryState state = getStateMachine().getCurrentState();
if (state == QueryState.QUERY_SUCCEEDED) {
return 1.0f;
} else {
int idx = 0;
- float [] subProgresses = new float[subqueries.size()];
+ List<SubQuery> tempSubQueries = new ArrayList<SubQuery>();
+ synchronized(subqueries) {
+ tempSubQueries.addAll(subqueries.values());
+ }
+ float [] subProgresses = new float[tempSubQueries.size()];
boolean finished = true;
- for (SubQuery subquery: subqueries.values()) {
+ for (SubQuery subquery: tempSubQueries) {
if (subquery.getState() != SubQueryState.NEW) {
subProgresses[idx] = subquery.getProgress();
if (finished && subquery.getState() != SubQueryState.SUCCEEDED) {
@@ -239,8 +232,8 @@ public class Query implements EventHandler<QueryEvent> {
public QueryId getId() {
return this.id;
}
-
- public SubQuery getSubQuery(SubQueryId id) {
+
+ public SubQuery getSubQuery(ExecutionBlockId id) {
return this.subqueries.get(id);
}
@@ -263,7 +256,7 @@ public class Query implements EventHandler<QueryEvent> {
@Override
public QueryState transition(Query query, QueryEvent queryEvent) {
query.setStartTime();
- query.context.setState(QueryState.QUERY_INIT);
+ //query.context.setState(QueryState.QUERY_INIT);
return QueryState.QUERY_INIT;
}
}
@@ -277,7 +270,8 @@ public class Query implements EventHandler<QueryEvent> {
query.sm);
subQuery.setPriority(query.priority--);
query.addSubQuery(subQuery);
- LOG.info("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+ LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
+
subQuery.handle(new SubQueryEvent(subQuery.getId(),
SubQueryEventType.SQ_INIT));
}
@@ -301,13 +295,16 @@ public class Query implements EventHandler<QueryEvent> {
query.addSubQuery(nextSubQuery);
nextSubQuery.handle(new SubQueryEvent(nextSubQuery.getId(),
SubQueryEventType.SQ_INIT));
- LOG.info("Scheduling SubQuery's Priority: " + nextSubQuery.getPriority());
- LOG.info("Scheduling SubQuery's Plan: \n" + nextSubQuery.getBlock().getPlan());
+ LOG.info("Scheduling SubQuery:" + nextSubQuery.getId());
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Scheduling SubQuery's Priority: " + nextSubQuery.getPriority());
+ LOG.debug("Scheduling SubQuery's Plan: \n" + nextSubQuery.getBlock().getPlan());
+ }
return query.checkQueryForCompleted();
} else { // Finish a query
if (query.checkQueryForCompleted() == QueryState.QUERY_SUCCEEDED) {
- SubQuery subQuery = query.getSubQuery(castEvent.getSubQueryId());
+ SubQuery subQuery = query.getSubQuery(castEvent.getExecutionBlockId());
TableDesc desc = new TableDescImpl(query.conf.getOutputTable(),
subQuery.getTableMeta(), query.context.getOutputPath());
query.setResultDesc(desc);
@@ -319,7 +316,7 @@ public class Query implements EventHandler<QueryEvent> {
query.eventHandler.handle(new QueryFinishEvent(query.getId()));
if (query.context.isCreateTableQuery()) {
- // TOOD move to QueryMasterManager
+ // TOOD move to QueryJobManager
//query.context.getCatalog().addTable(desc);
}
}
@@ -363,7 +360,6 @@ public class Query implements EventHandler<QueryEvent> {
public QueryState finished(QueryState finalState) {
setFinishTime();
- context.setState(finalState);
return finalState;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
new file mode 100644
index 0000000..2e2870f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
@@ -0,0 +1,285 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.master.rm.WorkerResourceManager;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.ProtoAsyncRpcClient;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class QueryInProgress extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryInProgress.class.getName());
+
+ private QueryId queryId;
+
+ private AsyncDispatcher dispatcher;
+
+ private LogicalRootNode plan;
+
+ private AtomicBoolean querySubmitted = new AtomicBoolean(false);
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ private QueryInfo queryInfo;
+
+ private final TajoMaster.MasterContext masterContext;
+
+ private ProtoAsyncRpcClient queryMasterRpc;
+
+ private TajoWorkerProtocol.TajoWorkerProtocolService queryMasterRpcClient;
+
+ public QueryInProgress(
+ TajoMaster.MasterContext masterContext,
+ QueryId queryId, String sql, LogicalRootNode plan) {
+ super(QueryInProgress.class.getName());
+ this.masterContext = masterContext;
+ this.queryId = queryId;
+ this.plan = plan;
+
+ queryInfo = new QueryInfo(queryId, sql);
+ queryInfo.setStartTime(System.currentTimeMillis());
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ dispatcher = new AsyncDispatcher();
+ this.addService(dispatcher);
+
+ dispatcher.register(QueryJobEvent.Type.class, new QueryInProgressEventHandler());
+ super.init(conf);
+ }
+
+ @Override
+ public void stop() {
+ synchronized(stopped) {
+ if(stopped.get()) {
+ return;
+ }
+ stopped.set(true);
+ }
+ LOG.info("=========================================================");
+ LOG.info("Stop query:" + queryId);
+
+ masterContext.getResourceManager().stopQueryMaster(queryId);
+
+ boolean queryMasterStopped = false;
+ long startTime = System.currentTimeMillis();
+ while(true) {
+ try {
+ if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
+ LOG.info("====> " + queryId + " QueryMaster stopped");
+ queryMasterStopped = true;
+ break;
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ break;
+ }
+ if(System.currentTimeMillis() - startTime > 60 * 1000) {
+ LOG.warn("Failed to stop QueryMaster:" + queryId);
+ break;
+ }
+ }
+
+ if(queryMasterRpc != null) {
+ //TODO release to connection pool
+ queryMasterRpc.close();
+ }
+ super.stop();
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ public EventHandler getEventHandler() {
+ return dispatcher.getEventHandler();
+ }
+
+ public void startQueryMaster() {
+ try {
+ LOG.info("Initializing QueryInProgress for QueryID=" + queryId);
+ WorkerResourceManager resourceManager = masterContext.getResourceManager();
+ WorkerResource queryMasterResource = resourceManager.allocateQueryMaster(this);
+
+ if(queryMasterResource != null) {
+ queryInfo.setQueryMasterResource(queryMasterResource);
+ }
+ getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInfo));
+ } catch (Exception e) {
+ catchException(e);
+ }
+ }
+
+ class QueryInProgressEventHandler implements EventHandler<QueryJobEvent> {
+ @Override
+ public void handle(QueryJobEvent queryJobEvent) {
+ if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_HEARTBEAT) {
+ heartbeat(queryJobEvent.getQueryInfo());
+ } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_MASTER_START) {
+ masterContext.getResourceManager().startQueryMaster(QueryInProgress.this);
+ } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_START) {
+ submmitQueryToMaster();
+ } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_FINISH) {
+ stop();
+ }
+ }
+ }
+
+ public TajoWorkerProtocol.TajoWorkerProtocolService getQueryMasterRpcClient() {
+ return queryMasterRpcClient;
+ }
+
+ private void connectQueryMaster() throws Exception {
+ if(queryInfo.getQueryMasterResource() != null &&
+ queryInfo.getQueryMasterResource().getAllocatedHost() != null) {
+ InetSocketAddress addr = NetUtils.createSocketAddr(
+ queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort());
+ LOG.info("Connect to QueryMaster:" + addr);
+ //TODO Get Connection from pool
+ queryMasterRpc = new ProtoAsyncRpcClient(TajoWorkerProtocol.class, addr);
+ queryMasterRpcClient = queryMasterRpc.getStub();
+ }
+ }
+
+ private synchronized void submmitQueryToMaster() {
+ if(querySubmitted.get()) {
+ return;
+ }
+
+ try {
+ if(queryMasterRpcClient == null) {
+ connectQueryMaster();
+ }
+ if(queryMasterRpcClient == null) {
+ LOG.info("No QueryMaster conneciton info.");
+ //TODO wait
+ return;
+ }
+ LOG.info("====>Call executeQuery to :" +
+ queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId);
+ queryMasterRpcClient.executeQuery(
+ null,
+ TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder()
+ .setQueryId(queryId.getProto())
+ .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build())
+ .build(), NullCallback.get());
+ querySubmitted.set(true);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ public void catchException(Exception e) {
+ LOG.error(e.getMessage(), e);
+ queryInfo.setQueryState(TajoProtos.QueryState.QUERY_FAILED);
+ queryInfo.setLastMessage(StringUtils.stringifyException(e));
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public QueryInfo getQueryInfo() {
+ return this.queryInfo;
+ }
+
+ private void heartbeat(QueryInfo queryInfo) {
+ LOG.info("Received QueryMaster heartbeat:" + queryInfo);
+ if(queryInfo.getQueryMasterResource() != null) {
+ this.queryInfo.setQueryMasterResource(queryInfo.getQueryMasterResource());
+ }
+ this.queryInfo.setQueryState(queryInfo.getQueryState());
+
+ if(queryInfo.getLastMessage() != null && !queryInfo.getLastMessage().isEmpty()) {
+ this.queryInfo.setLastMessage(queryInfo.getLastMessage());
+ LOG.info(queryId + queryInfo.getLastMessage());
+ }
+ if(this.queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_FAILED) {
+ //TODO needed QueryMaster's detail status(failed before or after launching worker)
+ //queryMasterStopped.set(true);
+ LOG.warn(queryId + " failed, " + queryInfo.getLastMessage());
+ }
+
+ if(!querySubmitted.get()) {
+ getEventHandler().handle(
+ new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, this.queryInfo));
+ }
+
+ if(isFinishState(this.queryInfo.getQueryState())) {
+ getEventHandler().handle(
+ new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_FINISH, this.queryInfo));
+ }
+ }
+
+ private boolean isFinishState(TajoProtos.QueryState state) {
+ return state == TajoProtos.QueryState.QUERY_FAILED ||
+ state == TajoProtos.QueryState.QUERY_KILLED ||
+ state == TajoProtos.QueryState.QUERY_SUCCEEDED;
+ }
+
+// private void checkQueryMasterShutdown() {
+// //run background
+// Thread t = new Thread() {
+// public void run() {
+// while(true) {
+// try {
+// if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
+// queryMasterStopped.set(true);
+// LOG.info("==========> " + queryId + " QueryMaster stopped");
+// break;
+// }
+// } catch (Exception e) {
+// LOG.error(e.getMessage(), e);
+// }
+// try {
+// Thread.sleep(1000);
+// } catch (InterruptedException e) {
+// break;
+// }
+// }
+// }
+// };
+//
+// t.start();
+// }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
new file mode 100644
index 0000000..e7ceae7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.master.rm.WorkerResource;
+
+public class QueryInfo {
+ private QueryId queryId;
+ private String sql;
+ private TajoProtos.QueryState queryState;
+ private float progress;
+ private long startTime;
+ private long finishTime;
+ private String lastMessage;
+ private WorkerResource queryMasterResource;
+
+ public QueryInfo(QueryId queryId) {
+ this(queryId, null);
+ }
+
+ public QueryInfo(QueryId queryId, String sql) {
+ this.queryId = queryId;
+ this.sql = sql;
+ this.queryState = TajoProtos.QueryState.QUERY_MASTER_INIT;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public String getSql() {
+ return sql;
+ }
+
+ public String getQueryMasterHost() {
+ if(queryMasterResource == null) {
+ return null;
+ }
+ return queryMasterResource.getAllocatedHost();
+ }
+
+ public void setQueryMasterResource(WorkerResource queryMasterResource) {
+ this.queryMasterResource = queryMasterResource;
+ }
+
+ public int getQueryMasterPort() {
+ if(queryMasterResource == null) {
+ return 0;
+ }
+ return queryMasterResource.getPorts()[0];
+ }
+
+ public int getQueryMasterClientPort() {
+ if(queryMasterResource == null) {
+ return 0;
+ }
+ return queryMasterResource.getPorts()[1];
+ }
+
+ public TajoProtos.QueryState getQueryState() {
+ return queryState;
+ }
+
+ public void setQueryState(TajoProtos.QueryState queryState) {
+ this.queryState = queryState;
+ }
+
+ public long getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime(long startTime) {
+ this.startTime = startTime;
+ }
+
+ public long getFinishTime() {
+ return finishTime;
+ }
+
+ public void setFinishTime(long finishTime) {
+ this.finishTime = finishTime;
+ }
+
+ public String getLastMessage() {
+ return lastMessage;
+ }
+
+ public void setLastMessage(String lastMessage) {
+ this.lastMessage = lastMessage;
+ }
+
+ public WorkerResource getQueryMasterResource() {
+ return queryMasterResource;
+ }
+
+ public float getProgress() {
+ return progress;
+ }
+
+ public void setProgress(float progress) {
+ this.progress = progress;
+ }
+
+ @Override
+ public String toString() {
+ return queryId.toString() + ", queryMaster=" + queryMasterResource;
+ }
+}
[3/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
new file mode 100644
index 0000000..8756eed
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnTajoResourceManager.java
@@ -0,0 +1,337 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.AMRMProtocol;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
+import org.apache.hadoop.yarn.api.records.*;
+import org.apache.hadoop.yarn.client.YarnClient;
+import org.apache.hadoop.yarn.client.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.YarnContainerProxy;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryJobEvent;
+import org.apache.tajo.util.ApplicationIdUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.security.PrivilegedAction;
+import java.util.*;
+
+public class YarnTajoResourceManager implements WorkerResourceManager {
+ private static final Log LOG = LogFactory.getLog(YarnTajoResourceManager.class);
+
+ private YarnClient yarnClient;
+ private AMRMProtocol rmClient;
+ private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+ private Configuration conf;
+ private TajoMaster.MasterContext masterContext;
+
+ public YarnTajoResourceManager() {
+
+ }
+
+ public YarnTajoResourceManager(TajoMaster.MasterContext masterContext) {
+ this.masterContext = masterContext;
+ }
+
+ @Override
+ public void stop() {
+ }
+
+ @Override
+ public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request) {
+ //nothing to do
+ //yarn manages worker membership.
+ }
+
+ @Override
+ public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource) {
+ //nothing to do
+ }
+
+ @Override
+ public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress) {
+ //nothing to do
+ //allocateAndLaunchQueryMaster in startQueryMaster()
+ return null;
+ }
+
+ @Override
+ public void allocateWorkerResources(
+ TajoMasterProtocol.WorkerResourceAllocationRequest request,
+ RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> rpcCallBack) {
+ //nothing to do
+ }
+
+ @Override
+ public void startQueryMaster(QueryInProgress queryInProgress) {
+ try {
+ allocateAndLaunchQueryMaster(queryInProgress);
+
+ queryInProgress.getEventHandler().handle(
+ new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
+
+ } catch (YarnRemoteException e) {
+ LOG.error(e.getMessage(), e);
+ //TODO set QueryState(fail)
+ }
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ this.conf = conf;
+ connectYarnClient();
+
+ final YarnConfiguration yarnConf = new YarnConfiguration(conf);
+ final YarnRPC rpc = YarnRPC.create(conf);
+ final InetSocketAddress rmAddress = conf.getSocketAddr(
+ YarnConfiguration.RM_SCHEDULER_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
+
+ UserGroupInformation currentUser;
+ try {
+ currentUser = UserGroupInformation.getCurrentUser();
+ } catch (IOException e) {
+ throw new YarnException(e);
+ }
+
+ rmClient = currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
+ @Override
+ public AMRMProtocol run() {
+ return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class, rmAddress, yarnConf);
+ }
+ });
+ }
+
+ @Override
+ public String getSeedQueryId() throws IOException {
+ try {
+ GetNewApplicationResponse newApp = yarnClient.getNewApplication();
+ ApplicationId appId = newApp.getApplicationId();
+
+ return appId.toString();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+
+ throw new IOException(e.getMessage(), e);
+ }
+ }
+
+ @Override
+ public void stopQueryMaster(QueryId queryId) {
+ try {
+ FinalApplicationStatus appStatus = FinalApplicationStatus.UNDEFINED;
+ QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+ if(queryInProgress != null) {
+ return;
+ }
+ TajoProtos.QueryState state = queryInProgress.getQueryInfo().getQueryState();
+ if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+ appStatus = FinalApplicationStatus.SUCCEEDED;
+ } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
+ appStatus = FinalApplicationStatus.FAILED;
+ } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
+ appStatus = FinalApplicationStatus.FAILED;
+ }
+ FinishApplicationMasterRequest request = recordFactory
+ .newRecordInstance(FinishApplicationMasterRequest.class);
+ request.setAppAttemptId(ApplicationIdUtils.createApplicationAttemptId(queryId));
+ request.setFinishApplicationStatus(appStatus);
+ request.setDiagnostics("QueryMaster shutdown by TajoMaster.");
+ rmClient.finishApplicationMaster(request);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ private void connectYarnClient() {
+ this.yarnClient = new YarnClientImpl();
+ this.yarnClient.init(conf);
+ this.yarnClient.start();
+ }
+
+ private ApplicationAttemptId allocateAndLaunchQueryMaster(QueryInProgress queryInProgress)
+ throws YarnRemoteException {
+ QueryId queryId = queryInProgress.getQueryId();
+ ApplicationId appId = ApplicationIdUtils.queryIdToAppId(queryId);
+
+ LOG.info("Allocate and launch ApplicationMaster for QueryMaster: queryId=" +
+ queryId + ", appId=" + appId);
+
+ ApplicationSubmissionContext appContext = Records.newRecord(ApplicationSubmissionContext.class);
+
+ // set the application id
+ appContext.setApplicationId(appId);
+ // set the application name
+ appContext.setApplicationName("Tajo");
+
+ Priority pri = Records.newRecord(Priority.class);
+ pri.setPriority(5);
+ appContext.setPriority(pri);
+
+ // Set the queue to which this application is to be submitted in the RM
+ appContext.setQueue("default");
+
+ ContainerLaunchContext commonContainerLaunchContext =
+ YarnContainerProxy.createCommonContainerLaunchContext(masterContext.getConf(), queryId.toString(), true);
+
+ // Setup environment by cloning from common env.
+ Map<String, String> env = commonContainerLaunchContext.getEnvironment();
+ Map<String, String> myEnv = new HashMap<String, String>(env.size());
+ myEnv.putAll(env);
+
+ ////////////////////////////////////////////////////////////////////////////
+ // Set the local resources
+ ////////////////////////////////////////////////////////////////////////////
+ // Set the necessary command to execute the application master
+ Vector<CharSequence> vargs = new Vector<CharSequence>(30);
+
+ // Set java executable command
+ //LOG.info("Setting up app master command");
+ vargs.add("${JAVA_HOME}" + "/bin/java");
+ // Set Xmx based on am memory size
+ String jvmOptions = masterContext.getConf().get("tajo.rm.yarn.querymaster.jvm.option", "-Xmx2000m");
+
+ for(String eachToken: jvmOptions.split((" "))) {
+ vargs.add(eachToken);
+ }
+ // Set Remote Debugging
+ //if (!context.getQuery().getSubQuery(event.getExecutionBlockId()).isLeafQuery()) {
+ //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
+ //}
+ // Set class name
+ //vargs.add(QueryMasterRunner.class.getCanonicalName());
+ vargs.add(TajoWorker.class.getCanonicalName());
+ vargs.add("qm");
+ vargs.add(queryId.toString()); // queryId
+ vargs.add(masterContext.getTajoMasterService().getBindAddress().getHostName() + ":" +
+ masterContext.getTajoMasterService().getBindAddress().getPort());
+
+ vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
+ vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
+
+ // Get final commmand
+ StringBuilder command = new StringBuilder();
+ for (CharSequence str : vargs) {
+ command.append(str).append(" ");
+ }
+
+ LOG.info("Completed setting up QueryMasterRunner command " + command.toString());
+ List<String> commands = new ArrayList<String>();
+ commands.add(command.toString());
+
+ final Resource resource = Records.newRecord(Resource.class);
+ // TODO - get default value from conf
+ resource.setMemory(256);
+ resource.setVirtualCores(1);
+
+ Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
+
+ ContainerLaunchContext masterContainerContext = BuilderUtils.newContainerLaunchContext(
+ null, commonContainerLaunchContext.getUser(),
+ resource, commonContainerLaunchContext.getLocalResources(), myEnv, commands,
+ myServiceData, null, new HashMap<ApplicationAccessType, String>(2));
+
+ appContext.setAMContainerSpec(masterContainerContext);
+
+ LOG.info("Submitting QueryMaster to ResourceManager");
+ yarnClient.submitApplication(appContext);
+
+ ApplicationReport appReport = monitorApplication(appId, EnumSet.of(YarnApplicationState.ACCEPTED));
+ ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
+
+ LOG.info("Launching QueryMaster with appAttemptId: " + attemptId);
+
+ return attemptId;
+ }
+
+ private ApplicationReport monitorApplication(ApplicationId appId,
+ Set<YarnApplicationState> finalState) throws YarnRemoteException {
+
+ long sleepTime = 100;
+ int count = 1;
+ while (true) {
+ // Get application report for the appId we are interested in
+ ApplicationReport report = yarnClient.getApplicationReport(appId);
+
+ LOG.info("Got application report from ASM for" + ", appId="
+ + appId.getId() + ", appAttemptId="
+ + report.getCurrentApplicationAttemptId() + ", clientToken="
+ + report.getClientToken() + ", appDiagnostics="
+ + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
+ + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
+ + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
+ + ", yarnAppState=" + report.getYarnApplicationState().toString()
+ + ", distributedFinalState="
+ + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
+ + report.getTrackingUrl() + ", appUser=" + report.getUser());
+
+ YarnApplicationState state = report.getYarnApplicationState();
+ if (finalState.contains(state)) {
+ return report;
+ }
+ try {
+ Thread.sleep(sleepTime);
+ sleepTime = count * 100;
+ if(count < 10) {
+ count++;
+ }
+ } catch (InterruptedException e) {
+ //LOG.debug("Thread sleep in monitoring loop interrupted");
+ }
+ }
+ }
+
+ public boolean isQueryMasterStopped(QueryId queryId) {
+ ApplicationId appId = ApplicationIdUtils.queryIdToAppId(queryId);
+ try {
+ ApplicationReport report = yarnClient.getApplicationReport(appId);
+ YarnApplicationState state = report.getYarnApplicationState();
+ return EnumSet.of(
+ YarnApplicationState.FINISHED,
+ YarnApplicationState.KILLED,
+ YarnApplicationState.FAILED).contains(state);
+ } catch (YarnRemoteException e) {
+ LOG.error(e.getMessage(), e);
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
new file mode 100644
index 0000000..5fe1b74
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/util/ApplicationIdUtils.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.util;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+
+public class ApplicationIdUtils {
+
+ public static ApplicationAttemptId createApplicationAttemptId(QueryId queryId) {
+ return BuilderUtils.newApplicationAttemptId(queryIdToAppId(queryId), 1);
+ }
+
+ public static ApplicationId queryIdToAppId(QueryId queryId) {
+ return BuilderUtils.newApplicationId(Long.parseLong(queryId.getId()), queryId.getSeq());
+ }
+
+ public static QueryId appIdToQueryId(YarnProtos.ApplicationIdProto appId) {
+ return QueryIdFactory.newQueryId(appId.getClusterTimestamp(), appId.getId());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
index f185cc6..b057561 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/webapp/StaticHttpServer.java
@@ -19,27 +19,24 @@
package org.apache.tajo.webapp;
import org.apache.hadoop.conf.Configuration;
-import org.mortbay.jetty.Connector;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.master.TajoMaster;
+import org.mortbay.jetty.Connector;
import java.io.IOException;
public class StaticHttpServer extends HttpServer {
private static StaticHttpServer instance = null;
- private TajoMaster master = null;
-
- private StaticHttpServer(TajoMaster master , String name, String bindAddress, int port,
+
+ private StaticHttpServer(Object containerObject , String name, String bindAddress, int port,
boolean findPort, Connector connector, Configuration conf,
String[] pathSpecs) throws IOException {
super( name, bindAddress, port, findPort, connector, conf, pathSpecs);
- this.master = master;
}
public static StaticHttpServer getInstance() {
return instance;
}
- public static StaticHttpServer getInstance( TajoMaster master, String name,
+ public static StaticHttpServer getInstance(Object containerObject, String name,
String bindAddress, int port, boolean findPort, Connector connector,
TajoConf conf,
String[] pathSpecs) throws IOException {
@@ -49,19 +46,16 @@ public class StaticHttpServer extends HttpServer {
addr = conf.getVar(ConfVars.TASKRUNNER_LISTENER_ADDRESS).split(":")[0];
}
- instance = new StaticHttpServer(master, name, addr, port,
+ instance = new StaticHttpServer(containerObject, name, addr, port,
findPort, connector, conf, pathSpecs);
- instance.setAttribute("tajo.master", master);
- instance.setAttribute("tajo.master.addr", addr);
- instance.setAttribute("tajo.master.conf", conf);
- instance.setAttribute("tajo.master.starttime", System.currentTimeMillis());
+ instance.setAttribute("tajo.info.server.object", containerObject);
+ instance.setAttribute("tajo.info.server.addr", addr);
+ instance.setAttribute("tajo.info.server.conf", conf);
+ instance.setAttribute("tajo.info.server.starttime", System.currentTimeMillis());
}
return instance;
}
- public TajoMaster getMaster() {
-
- return this.master;
- }
+
public void set(String name, Object object) {
instance.setAttribute(name, object);
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
new file mode 100644
index 0000000..ebc3f08
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/AbstractResourceAllocator.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.master.ContainerProxy;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class AbstractResourceAllocator extends CompositeService implements ResourceAllocator {
+ private int minCapability;
+ private int maxCapability;
+ private int numCluster;
+
+ private Map<ContainerId, ContainerProxy> containers = new HashMap<ContainerId, ContainerProxy>();
+
+ public AbstractResourceAllocator() {
+ super(AbstractResourceAllocator.class.getName());
+ }
+
+ public void addContainer(ContainerId cId, ContainerProxy container) {
+ containers.put(cId, container);
+ }
+
+ public void removeContainer(ContainerId cId) {
+ containers.remove(cId);
+ }
+
+ public boolean containsContainer(ContainerId cId) {
+ return containers.containsKey(cId);
+ }
+
+ public ContainerProxy getContainer(ContainerId cId) {
+ return containers.get(cId);
+ }
+
+ public Map<ContainerId, ContainerProxy> getContainers() {
+ return containers;
+ }
+
+ public void setMaxContainerCapability(int capability) {
+ this.maxCapability = capability;
+ }
+
+ public int getMaxContainerCapability() {
+ return this.maxCapability;
+ }
+
+ public void setMinContainerCapability(int capability) {
+ this.minCapability = capability;
+ }
+
+ public int getNumClusterNode() {
+ return numCluster;
+ }
+
+ public void setNumClusterNodes(int num) {
+ numCluster = num;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
new file mode 100644
index 0000000..108c7b7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/ResourceAllocator.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+
+public interface ResourceAllocator {
+ public void allocateTaskWorker();
+ public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerId);
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
new file mode 100644
index 0000000..3264f55
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -0,0 +1,346 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.proto.YarnProtos;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.ContainerProxy;
+import org.apache.tajo.master.TajoContainerProxy;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
+import org.apache.tajo.master.TaskRunnerLauncher;
+import org.apache.tajo.master.event.ContainerAllocationEvent;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.master.rm.TajoWorkerContainer;
+import org.apache.tajo.master.rm.TajoWorkerContainerId;
+import org.apache.tajo.master.rm.WorkerResource;
+import org.apache.tajo.rpc.CallFuture2;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class TajoResourceAllocator extends AbstractResourceAllocator {
+ private static final Log LOG = LogFactory.getLog(TajoResourceAllocator.class);
+
+ static AtomicInteger containerIdSeq = new AtomicInteger(0);
+ private TajoConf tajoConf;
+ private QueryMasterTask.QueryContext queryContext;
+ private final ExecutorService executorService;
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ public TajoResourceAllocator(QueryMasterTask.QueryContext queryContext) {
+ this.queryContext = queryContext;
+ executorService = Executors.newFixedThreadPool(
+ queryContext.getConf().getIntVar(TajoConf.ConfVars.AM_TASKRUNNER_LAUNCH_PARALLEL_NUM));
+ }
+
+ @Override
+ public ContainerId makeContainerId(YarnProtos.ContainerIdProto containerIdProto) {
+ TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+ ApplicationAttemptId appAttemptId = new ApplicationAttemptIdPBImpl(containerIdProto.getAppAttemptId());
+ containerId.setApplicationAttemptId(appAttemptId);
+ containerId.setId(containerIdProto.getId());
+ return containerId;
+ }
+
+ @Override
+ public void allocateTaskWorker() {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ tajoConf = (TajoConf)conf;
+
+ queryContext.getDispatcher().register(TaskRunnerGroupEvent.EventType.class, new TajoTaskRunnerLauncher());
+//
+ queryContext.getDispatcher().register(ContainerAllocatorEventType.class, new TajoWorkerAllocationHandler());
+
+ super.init(conf);
+ }
+
+ @Override
+ public void stop() {
+ if(stopped.get()) {
+ return;
+ }
+ stopped.set(true);
+ executorService.shutdownNow();
+
+ while(!executorService.isTerminated()) {
+ LOG.info("====>executorService.isTerminated:" + executorService.isTerminated() + "," +
+ executorService.isShutdown());
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ }
+ }
+ Map<ContainerId, ContainerProxy> containers = queryContext.getResourceAllocator().getContainers();
+ for(ContainerProxy eachProxy: containers.values()) {
+ try {
+ eachProxy.stopContainer();
+ } catch (Exception e) {
+ }
+ }
+ super.stop();
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ final public static FsPermission QUERYCONF_FILE_PERMISSION =
+ FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+ private static void writeConf(Configuration conf, Path queryConfFile)
+ throws IOException {
+ // Write job file to Tajo's fs
+ FileSystem fs = queryConfFile.getFileSystem(conf);
+ FSDataOutputStream out =
+ FileSystem.create(fs, queryConfFile,
+ new FsPermission(QUERYCONF_FILE_PERMISSION));
+ try {
+ conf.writeXml(out);
+ } finally {
+ out.close();
+ }
+ }
+
+ class TajoTaskRunnerLauncher implements TaskRunnerLauncher {
+ @Override
+ public void handle(TaskRunnerGroupEvent event) {
+ if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_LAUNCH) {
+ launchTaskRunners(event.getExecutionBlockId(), event.getContainers());
+ } else if (event.getType() == TaskRunnerGroupEvent.EventType.CONTAINER_REMOTE_CLEANUP) {
+ stopContainers(event.getContainers());
+ }
+ }
+ }
+
+ private void launchTaskRunners(ExecutionBlockId executionBlockId, Collection<Container> containers) {
+ FileSystem fs = null;
+
+ QueryConf queryConf = queryContext.getConf();
+ LOG.info("defaultFS: " + queryConf.get("fs.default.name"));
+ LOG.info("defaultFS: " + queryConf.get("fs.defaultFS"));
+ try {
+ fs = FileSystem.get(queryConf);
+ } catch (IOException e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ try {
+ // TODO move to tajo temp
+ Path warehousePath = new Path(queryConf.getVar(TajoConf.ConfVars.ROOT_DIR), TajoConstants.WAREHOUSE_DIR);
+ Path queryConfPath = new Path(warehousePath, executionBlockId.getQueryId().toString());
+ queryConfPath = new Path(queryConfPath, QueryConf.FILENAME);
+
+ if(!fs.exists(queryConfPath)){
+ LOG.info("Writing a QueryConf to HDFS and add to local environment, outputPath=" + queryConf.getOutputPath());
+ writeConf(queryConf, queryConfPath);
+ } else {
+ LOG.warn("QueryConf already exist. path: " + queryConfPath.toString());
+ }
+ } catch (IOException e) {
+ LOG.error(e.getMessage(), e);
+ }
+ //Query in standby mode doesn't need launch Worker.
+ //But, Assign ExecutionBlock to assigned tajo worker
+ for(Container eachContainer: containers) {
+ TajoContainerProxy containerProxy = new TajoContainerProxy(queryContext, tajoConf,
+ eachContainer, executionBlockId);
+ executorService.submit(new LaunchRunner(eachContainer.getId(), containerProxy));
+ }
+ }
+
+ protected class LaunchRunner implements Runnable {
+ private final ContainerProxy proxy;
+ private final ContainerId id;
+ public LaunchRunner(ContainerId id, ContainerProxy proxy) {
+ this.proxy = proxy;
+ this.id = id;
+ }
+ @Override
+ public void run() {
+ proxy.launch(null);
+ LOG.info("ContainerProxy started:" + id);
+ }
+ }
+
+ private void stopContainers(Collection<Container> containers) {
+ for (Container container : containers) {
+ final ContainerProxy proxy = queryContext.getResourceAllocator().getContainer(container.getId());
+ executorService.submit(new StopContainerRunner(container.getId(), proxy));
+ }
+ }
+
+ private class StopContainerRunner implements Runnable {
+ private final ContainerProxy proxy;
+ private final ContainerId id;
+ public StopContainerRunner(ContainerId id, ContainerProxy proxy) {
+ this.id = id;
+ this.proxy = proxy;
+ }
+
+ @Override
+ public void run() {
+ LOG.info("ContainerProxy stopped:" + id + "," + proxy.getId());
+ proxy.stopContainer();
+ }
+ }
+
+ class TajoWorkerAllocationHandler implements EventHandler<ContainerAllocationEvent> {
+ @Override
+ public void handle(ContainerAllocationEvent event) {
+ executorService.submit(new TajoWorkerAllocationThread(event));
+ }
+ }
+
+ class TajoWorkerAllocationThread extends Thread {
+ ContainerAllocationEvent event;
+ TajoWorkerAllocationThread(ContainerAllocationEvent event) {
+ this.event = event;
+ }
+
+ @Override
+ public void run() {
+ LOG.info("======> Start TajoWorkerAllocationThread");
+ CallFuture2<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack =
+ new CallFuture2<TajoMasterProtocol.WorkerResourceAllocationResponse>();
+
+ int requiredMemoryMBSlot = 512; //TODO
+ int requiredDiskSlots = 1; //TODO
+ TajoMasterProtocol.WorkerResourceAllocationRequest request =
+ TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+ .setMemoryMBSlots(requiredMemoryMBSlot)
+ .setDiskSlots(requiredDiskSlots)
+ .setNumWorks(event.getRequiredNum())
+ .setExecutionBlockId(event.getExecutionBlockId().getProto())
+ .build();
+
+ queryContext.getQueryMasterContext().getWorkerContext().
+ getTajoMasterRpcClient().allocateWorkerResources(null, request, callBack);
+
+ int numAllocatedWorkers = 0;
+ while(!stopped.get()) {
+ TajoMasterProtocol.WorkerResourceAllocationResponse response = null;
+ try {
+ response = callBack.get(3, TimeUnit.SECONDS);
+ } catch (InterruptedException e) {
+ if(stopped.get()) {
+ break;
+ }
+ } catch (TimeoutException e) {
+ LOG.info("No available worker resource for " + event.getExecutionBlockId());
+ continue;
+ }
+
+ List<String> workerHosts = response.getAllocatedWorksList();
+ ExecutionBlockId executionBlockId = event.getExecutionBlockId();
+
+ List<Container> containers = new ArrayList<Container>();
+ for(String eachWorker: workerHosts) {
+ TajoWorkerContainer container = new TajoWorkerContainer();
+ NodeIdPBImpl nodeId = new NodeIdPBImpl();
+ String[] tokens = eachWorker.split(":");
+
+ nodeId.setHost(tokens[0]);
+ nodeId.setPort(Integer.parseInt(tokens[1]));
+
+ TajoWorkerContainerId containerId = new TajoWorkerContainerId();
+
+ containerId.setApplicationAttemptId(
+ ApplicationIdUtils.createApplicationAttemptId(executionBlockId.getQueryId()));
+ containerId.setId(containerIdSeq.incrementAndGet());
+
+ container.setId(containerId);
+ container.setNodeId(nodeId);
+
+ WorkerResource workerResource = new WorkerResource();
+ workerResource.setAllocatedHost(nodeId.getHost());
+ workerResource.setPorts(new int[]{nodeId.getPort()});
+ workerResource.setMemoryMBSlots(requiredMemoryMBSlot);
+ workerResource.setDiskSlots(requiredDiskSlots);
+
+ container.setWorkerResource(workerResource);
+
+ containers.add(container);
+ }
+
+ SubQueryState state = queryContext.getSubQuery(executionBlockId).getState();
+ if (!SubQuery.isRunningState(state)) {
+ List<WorkerResource> workerResources = new ArrayList<WorkerResource>();
+ for(Container eachContainer: containers) {
+ workerResources.add(((TajoWorkerContainer)eachContainer).getWorkerResource());
+ }
+ try {
+ TajoContainerProxy.releaseWorkerResource(queryContext, executionBlockId, workerResources);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ return;
+ }
+
+ if (workerHosts.size() > 0) {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("SubQueryContainerAllocationEvent fire:" + executionBlockId);
+ }
+ queryContext.getEventHandler().handle(new SubQueryContainerAllocationEvent(executionBlockId, containers));
+ }
+ numAllocatedWorkers += workerHosts.size();
+ if(numAllocatedWorkers >= event.getRequiredNum()) {
+ break;
+ }
+ }
+ LOG.info("======> Stop TajoWorkerAllocationThread");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
new file mode 100644
index 0000000..3b116f2
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -0,0 +1,424 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryMaster;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.pullserver.TajoPullServerService;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.ProtoAsyncRpcClient;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.webapp.StaticHttpServer;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.InputStreamReader;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TajoWorker extends CompositeService {
+ public static PrimitiveProtos.BoolProto TRUE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+ public static PrimitiveProtos.BoolProto FALSE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+ private static final Log LOG = LogFactory.getLog(TajoWorker.class);
+
+ private TajoConf tajoConf;
+
+ private StaticHttpServer webServer;
+
+ private TajoWorkerClientService tajoWorkerClientService;
+
+ private TajoWorkerManagerService tajoWorkerManagerService;
+
+ //to TajoMaster
+ private ProtoAsyncRpcClient tajoMasterRpc;
+
+ private TajoMasterProtocol.TajoMasterProtocolService tajoMasterRpcClient;
+
+ private WorkerContext workerContext;
+
+ private TaskRunnerManager taskRunnerManager;
+
+ private TajoPullServerService pullService;
+
+ private String daemonMode;
+
+ private WorkerHeartbeatThread workerHeartbeatThread;
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ public TajoWorker(String daemonMode) throws Exception {
+ super(TajoWorker.class.getName());
+ this.daemonMode = daemonMode;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
+
+ this.tajoConf = (TajoConf)conf;
+ RackResolver.init(tajoConf);
+
+ workerContext = new WorkerContext();
+
+ String resourceManagerClassName = conf.get("tajo.resource.manager",
+ TajoWorkerResourceManager.class.getCanonicalName());
+
+ boolean randomPort = true;
+ if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+ randomPort = false;
+ }
+ int infoPort = tajoConf.getInt("tajo.worker.info.port", 8090);
+ int clientPort = tajoConf.getInt("tajo.worker.client.rpc.port", 8091);
+ int managerPort = tajoConf.getInt("tajo.worker.manager.rpc.port", 8092);
+
+ if(randomPort) {
+ clientPort = 0;
+ managerPort = 0;
+ tajoConf.setInt(TajoConf.ConfVars.PULLSERVER_PORT.varname, 0);
+ //infoPort = 0;
+ }
+ try {
+ //TODO WebServer port configurable
+ webServer = StaticHttpServer.getInstance(this, "admin", null, infoPort,
+ true, null, tajoConf, null);
+ webServer.start();
+ } catch (Exception e) {
+ LOG.error("Can' start info http server:" + e.getMessage(), e);
+ }
+
+ if(!"qm".equals(daemonMode)) {
+ taskRunnerManager = new TaskRunnerManager(workerContext);
+ addService(taskRunnerManager);
+ }
+
+ if(workerContext.isStandbyMode()) {
+ pullService = new TajoPullServerService();
+ addService(pullService);
+ }
+
+ if(!"tr".equals(daemonMode)) {
+ tajoWorkerClientService = new TajoWorkerClientService(workerContext, clientPort);
+ addService(tajoWorkerClientService);
+
+ tajoWorkerManagerService = new TajoWorkerManagerService(workerContext, managerPort);
+ addService(tajoWorkerManagerService);
+ LOG.info("====> Tajo worker started: mode=" + daemonMode + ", clientPort=" + clientPort + ", managerPort=" + managerPort);
+ } else {
+ LOG.info("====> Tajo worker started: mode=" + daemonMode);
+ }
+
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(stopped.get()) {
+ return;
+ }
+ stopped.set(true);
+ if(webServer != null) {
+ try {
+ webServer.stop();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+ if(workerHeartbeatThread != null) {
+ workerHeartbeatThread.interrupt();
+ }
+
+// try {
+// FileSystem.closeAll();
+// } catch (IOException e) {
+// LOG.error(e.getMessage(), e);
+// }
+ if(tajoMasterRpc != null) {
+ tajoMasterRpc.close();
+ }
+// for(Service eachService: getServices()) {
+// System.out.println("Service:" + eachService);
+// }
+ super.stop();
+ LOG.info("TajoWorker main thread exiting");
+ }
+
+ public class WorkerContext {
+ public QueryMaster getQueryMaster() {
+ return tajoWorkerManagerService.getQueryMaster();
+ }
+
+ public TajoWorkerManagerService getTajoWorkerManagerService() {
+ return tajoWorkerManagerService;
+ }
+
+ public TajoWorkerClientService getTajoWorkerClientService() {
+ return tajoWorkerClientService;
+ }
+
+ public TajoMasterProtocol.TajoMasterProtocolService getTajoMasterRpcClient() {
+ return tajoMasterRpcClient;
+ }
+
+ public TaskRunnerManager getTaskRunnerManager() {
+ return taskRunnerManager;
+ }
+
+ public TajoPullServerService getPullService() {
+ return pullService;
+ }
+
+ public void stopWorker(boolean force) {
+ stop();
+ if(force) {
+ System.exit(0);
+ }
+ }
+
+ public boolean isStandbyMode() {
+ return !"qm".equals(daemonMode) && !"tr".equals(daemonMode);
+ }
+ }
+
+ public void stopWorkerForce() {
+ stop();
+ }
+
+ private void setWorkerMode(String[] params) {
+ if("qm".equals(daemonMode)) {
+ //QueryMaster mode
+ String tajoMasterAddress = params[2];
+
+ LOG.info("Init TajoMaster connection to:" + tajoMasterAddress);
+ InetSocketAddress addr = NetUtils.createSocketAddr(tajoMasterAddress);
+ try {
+ tajoMasterRpc = new ProtoAsyncRpcClient(TajoMasterProtocol.class, addr);
+ tajoMasterRpcClient = tajoMasterRpc.getStub();
+ } catch (Exception e) {
+ LOG.error("Can't connect to TajoMaster[" + addr + "], " + e.getMessage(), e);
+ }
+
+ QueryId queryId = TajoIdUtils.parseQueryId(params[1]);
+ tajoWorkerManagerService.getQueryMaster().reportQueryStatusToQueryMaster(
+ queryId, TajoProtos.QueryState.QUERY_MASTER_LAUNCHED);
+ } else if("tr".equals(daemonMode)) {
+ //TaskRunner mode
+ taskRunnerManager.startTask(params);
+ } else {
+ //Standby mode
+ String tajoMasterAddress = tajoConf.get("tajo.master.manager.addr");
+ LOG.info("Init TajoMaster connection to:" + tajoMasterAddress);
+ InetSocketAddress addr = NetUtils.createSocketAddr(tajoMasterAddress);
+ try {
+ tajoMasterRpc = new ProtoAsyncRpcClient(TajoMasterProtocol.class, addr);
+ tajoMasterRpcClient = tajoMasterRpc.getStub();
+ } catch (Exception e) {
+ LOG.error("Can't connect to TajoMaster[" + addr + "], " + e.getMessage(), e);
+ }
+ workerHeartbeatThread = new WorkerHeartbeatThread();
+ workerHeartbeatThread.start();
+ }
+ }
+
+ class WorkerHeartbeatThread extends Thread {
+ TajoMasterProtocol.ServerStatusProto.System systemInfo;
+ List<TajoMasterProtocol.ServerStatusProto.Disk> diskInfos =
+ new ArrayList<TajoMasterProtocol.ServerStatusProto.Disk>();
+ int workerDiskSlots;
+ List<File> mountPaths;
+
+ public WorkerHeartbeatThread() {
+ int workerMemoryMBSlots;
+ int workerCpuCoreSlots;
+
+ boolean useSystemInfo = tajoConf.getBoolean("tajo.worker.slots.use.os.info", true);
+
+ try {
+ mountPaths = getMountPath();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ if(useSystemInfo) {
+ float memoryRatio = tajoConf.getFloat("tajo.worker.slots.os.memory.ratio", 0.8f);
+ workerMemoryMBSlots = getTotalMemoryMB();
+ workerMemoryMBSlots = (int)((float)(workerMemoryMBSlots) * memoryRatio);
+ workerCpuCoreSlots = Runtime.getRuntime().availableProcessors();
+ if(mountPaths == null) {
+ workerDiskSlots = 2;
+ } else {
+ workerDiskSlots = mountPaths.size();
+ }
+ } else {
+ workerMemoryMBSlots = tajoConf.getInt("tajo.worker.slots.memoryMB", 2048);
+ workerDiskSlots = tajoConf.getInt("tajo.worker.slots.disk", 2);
+ workerCpuCoreSlots = tajoConf.getInt("tajo.worker.slots.cpu.core", 4);
+ }
+
+ workerDiskSlots = workerDiskSlots * tajoConf.getInt("tajo.worker.slots.disk.concurrency", 4);
+
+ systemInfo = TajoMasterProtocol.ServerStatusProto.System.newBuilder()
+ .setAvailableProcessors(workerCpuCoreSlots)
+ .setFreeMemoryMB(0)
+ .setMaxMemoryMB(0)
+ .setTotalMemoryMB(workerMemoryMBSlots)
+ .build();
+ }
+
+ public void run() {
+ LOG.info("Worker Resource Heartbeat Thread start.");
+ int sendDiskInfoCount = 0;
+ while(true) {
+ if(sendDiskInfoCount == 0 && mountPaths != null) {
+ for(File eachFile: mountPaths) {
+ diskInfos.clear();
+ diskInfos.add(TajoMasterProtocol.ServerStatusProto.Disk.newBuilder()
+ .setAbsolutePath(eachFile.getAbsolutePath())
+ .setTotalSpace(eachFile.getTotalSpace())
+ .setFreeSpace(eachFile.getFreeSpace())
+ .setUsableSpace(eachFile.getUsableSpace())
+ .build());
+ }
+ }
+ TajoMasterProtocol.ServerStatusProto serverStatus = TajoMasterProtocol.ServerStatusProto.newBuilder()
+ .addAllDisk(diskInfos)
+ .setRunningTaskNum(0) //TODO
+ .setSystem(systemInfo)
+ .setDiskSlots(workerDiskSlots)
+ .build();
+
+ TajoMasterProtocol.TajoHeartbeat heartbeatProto = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+ .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+ .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+ .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+ .setServerStatus(serverStatus)
+ .build();
+
+ workerContext.getTajoMasterRpcClient().heartbeat(null, heartbeatProto, NullCallback.get());
+ try {
+ Thread.sleep(10 * 1000);
+ } catch (InterruptedException e) {
+ break;
+ }
+ sendDiskInfoCount++;
+
+ if(sendDiskInfoCount > 10) {
+ sendDiskInfoCount = 0;
+ }
+ }
+
+ LOG.info("Worker Resource Heartbeat Thread stopped.");
+ }
+ }
+
+ private class ShutdownHook implements Runnable {
+ @Override
+ public void run() {
+ if(!stopped.get()) {
+ LOG.info("============================================");
+ LOG.info("TajoWorker received SIGINT Signal");
+ LOG.info("============================================");
+ stop();
+ }
+ }
+ }
+
+ public void startWorker(TajoConf tajoConf, String[] args) {
+ init(tajoConf);
+ start();
+ setWorkerMode(args);
+ }
+
+ public static List<File> getMountPath() throws Exception {
+ BufferedReader mountOutput = null;
+ try {
+ Process mountProcess = Runtime.getRuntime ().exec("mount");
+ mountOutput = new BufferedReader(new InputStreamReader(mountProcess.getInputStream()));
+ List<File> mountPaths = new ArrayList<File>();
+ while (true) {
+ String line = mountOutput.readLine();
+ if (line == null) {
+ break;
+ }
+
+ System.out.println(line);
+
+ int indexStart = line.indexOf(" on /");
+ int indexEnd = line.indexOf(" ", indexStart + 4);
+
+ mountPaths.add(new File(line.substring (indexStart + 4, indexEnd)));
+ }
+ return mountPaths;
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ } finally {
+ if(mountOutput != null) {
+ mountOutput.close();
+ }
+ }
+ }
+
+ public static int getTotalMemoryMB() {
+ com.sun.management.OperatingSystemMXBean bean =
+ (com.sun.management.OperatingSystemMXBean)
+ java.lang.management.ManagementFactory.getOperatingSystemMXBean();
+ long max = bean.getTotalPhysicalMemorySize();
+ return ((int)(max/1024));
+ }
+
+ public static void main(String[] args) throws Exception {
+ StringUtils.startupShutdownMessage(TajoWorker.class, args, LOG);
+
+ if(args.length < 1) {
+ LOG.error("Wrong startup params");
+ System.exit(-1);
+ }
+
+ String workerMode = args[0];
+
+ try {
+ TajoWorker tajoWorker = new TajoWorker(workerMode);
+ tajoWorker.startWorker(new TajoConf(new YarnConfiguration()), args);
+ } catch (Throwable t) {
+ LOG.fatal("Error starting TajoWorker", t);
+ System.exit(-1);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
new file mode 100644
index 0000000..3969b36
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.QueryMasterClientProtocol;
+import org.apache.tajo.master.querymaster.Query;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.rpc.ProtoBlockingRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+
+public class TajoWorkerClientService extends AbstractService {
+ private static final Log LOG = LogFactory.getLog(TajoWorkerClientService.class);
+ private final PrimitiveProtos.BoolProto BOOL_TRUE =
+ PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
+ private final PrimitiveProtos.BoolProto BOOL_FALSE =
+ PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
+ private ProtoBlockingRpcServer rpcServer;
+ private InetSocketAddress bindAddr;
+ private String addr;
+ private int port;
+ private Configuration conf;
+ private TajoWorker.WorkerContext workerContext;
+ private TajoWorkerClientProtocolServiceHandler serviceHandler;
+
+ public TajoWorkerClientService(TajoWorker.WorkerContext workerContext, int port) {
+ super(TajoWorkerClientService.class.getName());
+
+ this.port = port;
+ this.workerContext = workerContext;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ this.conf = conf;
+ this.serviceHandler = new TajoWorkerClientProtocolServiceHandler();
+
+ // init RPC Server in constructor cause Heartbeat Thread use bindAddr
+ // Setup RPC server
+ try {
+ // TODO initial port num is value of config and find unused port with sequence
+ InetSocketAddress initIsa = new InetSocketAddress("0.0.0.0", port);
+ if (initIsa.getAddress() == null) {
+ throw new IllegalArgumentException("Failed resolve of " + initIsa);
+ }
+
+ // TODO blocking/non-blocking??
+ this.rpcServer = new ProtoBlockingRpcServer(QueryMasterClientProtocol.class, serviceHandler, initIsa);
+ this.rpcServer.start();
+
+ this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+ this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+ this.port = bindAddr.getPort();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ // Get the master address
+ LOG.info(TajoWorkerClientService.class.getSimpleName() + " is bind to " + addr);
+ //queryConf.setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
+
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ LOG.info("====> TajoWorkerClientService stopping");
+ if(rpcServer != null) {
+ rpcServer.shutdown();
+ }
+ LOG.info("====> TajoWorkerClientService stopped");
+ super.stop();
+ }
+
+ public InetSocketAddress getBindAddr() {
+ return bindAddr;
+ }
+
+ public class TajoWorkerClientProtocolServiceHandler
+ implements QueryMasterClientProtocol.QueryMasterClientProtocolService.BlockingInterface {
+ @Override
+ public PrimitiveProtos.BoolProto updateSessionVariables(
+ RpcController controller,
+ ClientProtos.UpdateSessionVariableRequest request) throws ServiceException {
+ return null;
+ }
+
+ @Override
+ public ClientProtos.GetQueryResultResponse getQueryResult(
+ RpcController controller,
+ ClientProtos.GetQueryResultRequest request) throws ServiceException {
+ QueryId queryId = new QueryId(request.getQueryId());
+ Query query = workerContext.getQueryMaster().getQuery(queryId);
+
+ ClientProtos.GetQueryResultResponse.Builder builder = ClientProtos.GetQueryResultResponse.newBuilder();
+
+ if(query == null) {
+ builder.setErrorMessage("No Query for " + queryId);
+ } else {
+ switch (query.getState()) {
+ case QUERY_SUCCEEDED:
+ builder.setTableDesc((CatalogProtos.TableDescProto)query.getResultDesc().getProto());
+ break;
+ case QUERY_FAILED:
+ case QUERY_ERROR:
+ builder.setErrorMessage("Query " + queryId + " is failed");
+ default:
+ builder.setErrorMessage("Query " + queryId + " is still running");
+ }
+ }
+ return builder.build();
+ }
+
+ @Override
+ public ClientProtos.GetQueryStatusResponse getQueryStatus(
+ RpcController controller,
+ ClientProtos.GetQueryStatusRequest request) throws ServiceException {
+ ClientProtos.GetQueryStatusResponse.Builder builder
+ = ClientProtos.GetQueryStatusResponse.newBuilder();
+ QueryId queryId = new QueryId(request.getQueryId());
+
+ builder.setQueryId(request.getQueryId());
+
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
+ builder.setResultCode(ClientProtos.ResultCode.OK);
+ builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
+ } else {
+ QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(queryId);
+ builder.setResultCode(ClientProtos.ResultCode.OK);
+ builder.setQueryMasterHost(bindAddr.getHostName());
+ builder.setQueryMasterPort(bindAddr.getPort());
+
+ if (queryMasterTask != null) {
+ queryMasterTask.touchSessionTime();
+ Query query = queryMasterTask.getQuery();
+
+ builder.setState(query.getState());
+ builder.setProgress(query.getProgress());
+ builder.setSubmitTime(query.getAppSubmitTime());
+ builder.setInitTime(query.getInitializationTime());
+ builder.setHasResult(!queryMasterTask.getQueryContext().isCreateTableQuery());
+ if (query.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+ builder.setFinishTime(query.getFinishTime());
+ } else {
+ builder.setFinishTime(System.currentTimeMillis());
+ }
+ } else {
+ builder.setState(TajoProtos.QueryState.QUERY_NOT_ASSIGNED);
+ }
+ }
+
+ return builder.build();
+ }
+
+ @Override
+ public PrimitiveProtos.BoolProto killQuery (
+ RpcController controller,
+ TajoIdProtos.QueryIdProto request) throws ServiceException {
+ final QueryId queryId = new QueryId(request);
+ LOG.info("Stop Query:" + queryId);
+ Thread t = new Thread() {
+ public void run() {
+// try {
+// Thread.sleep(1000); //wait tile return to rpc response
+// } catch (InterruptedException e) {
+// }
+ workerContext.getQueryMaster().getContext().stopQuery(queryId);
+ }
+ };
+ t.start();
+ return BOOL_TRUE;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
new file mode 100644
index 0000000..a48339a
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.worker;
+
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryUnitAttemptId;
+import org.apache.tajo.TajoIdProtos;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TaskSchedulerImpl;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.querymaster.QueryMaster;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.rpc.ProtoAsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+
+public class TajoWorkerManagerService extends CompositeService
+ implements TajoWorkerProtocol.TajoWorkerProtocolService.Interface {
+ private static final Log LOG = LogFactory.getLog(TajoWorkerManagerService.class.getName());
+
+ private ProtoAsyncRpcServer rpcServer;
+ private InetSocketAddress bindAddr;
+ private String addr;
+ private int port;
+
+ private QueryMaster queryMaster;
+
+ private TajoWorker.WorkerContext workerContext;
+
+ public TajoWorkerManagerService(TajoWorker.WorkerContext workerContext, int port) {
+ super(TajoWorkerManagerService.class.getName());
+ this.workerContext = workerContext;
+ this.port = port;
+ }
+
+ public QueryMaster getQueryMaster() {
+ return queryMaster;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ try {
+ // Setup RPC server
+ InetSocketAddress initIsa =
+ new InetSocketAddress("0.0.0.0", port);
+ if (initIsa.getAddress() == null) {
+ throw new IllegalArgumentException("Failed resolve of " + initIsa);
+ }
+
+ this.rpcServer = new ProtoAsyncRpcServer(TajoWorkerProtocol.class, this, initIsa);
+ this.rpcServer.start();
+
+ this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+ this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+ this.port = bindAddr.getPort();
+
+ queryMaster = new QueryMaster(workerContext);
+ addService(queryMaster);
+
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ // Get the master address
+ LOG.info("TajoWorkerManagerService is bind to " + addr);
+ ((TajoConf)conf).setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
+
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(rpcServer != null) {
+ rpcServer.shutdown();
+ }
+ LOG.info("TajoWorkerManagerService stopped");
+ super.stop();
+ }
+
+ public InetSocketAddress getBindAddr() {
+ return bindAddr;
+ }
+
+ public String getHostAndPort() {
+ return bindAddr.getHostName() + ":" + bindAddr.getPort();
+ }
+
+ @Override
+ public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestProto request,
+ RpcCallback<TajoWorkerProtocol.QueryUnitRequestProto> done) {
+ try {
+ ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId());
+ QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(ebId.getQueryId());
+ ContainerId cid =
+ queryMasterTask.getQueryContext().getResourceAllocator().makeContainerId(request.getContainerId());
+
+ if(queryMasterTask == null || queryMasterTask.isStopped()) {
+ LOG.info("====>getTask:" + cid + ", ebId:" + ebId + ", but query is finished.");
+ done.run(TaskSchedulerImpl.stopTaskRunnerReq);
+ } else {
+ LOG.info("====>getTask:" + cid + ", ebId:" + ebId);
+ queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(cid, ebId, done));
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ @Override
+ public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+ new QueryId(request.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+ queryMasterTask.getEventHandler().handle(
+ new TaskAttemptStatusUpdateEvent(new QueryUnitAttemptId(request.getId()), request));
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void ping(RpcController controller,
+ TajoIdProtos.QueryUnitAttemptIdProto attemptId,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ // TODO - to be completed
+// QueryUnitAttemptId attemptId = new QueryUnitAttemptId(attemptIdProto);
+// context.getQuery(attemptId.getQueryId()).getSubQuery(attemptId.getExecutionBlockId()).
+// getQueryUnit(attemptId.getQueryUnitId()).getAttempt(attemptId).
+// resetExpireTime();
+ done.run(TajoWorker.TRUE_PROTO);
+ }
+
+ @Override
+ public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErrorReport report,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+ new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+ queryMasterTask.getEventHandler().handle(new TaskFatalErrorEvent(report));
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionReport report,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+ new QueryId(report.getId().getQueryUnitId().getExecutionBlockId().getQueryId()));
+ queryMasterTask.getEventHandler().handle(new TaskCompletionEvent(report));
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void executeQuery(RpcController controller,
+ TajoWorkerProtocol.QueryExecutionRequestProto request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryId queryId = new QueryId(request.getQueryId());
+ LOG.info("====>Receive executeQuery request:" + queryId);
+ queryMaster.handle(new QueryStartEvent(queryId, request.getLogicalPlanJson().getValue()));
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void executeExecutionBlock(RpcController controller,
+ TajoWorkerProtocol.RunExecutionBlockRequestProto request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ String[] params = new String[7];
+ params[0] = "standby"; //mode(never used)
+ params[1] = request.getExecutionBlockId();
+ // NodeId has a form of hostname:port.
+ params[2] = request.getNodeId();
+ params[3] = request.getContainerId();
+
+ // QueryMaster's address
+ params[4] = request.getQueryMasterHost();
+ params[5] = String.valueOf(request.getQueryMasterPort());
+ params[6] = request.getQueryOutputPath();
+ workerContext.getTaskRunnerManager().startTask(params);
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
index 94f1662..f9cc82c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
@@ -37,7 +37,6 @@ import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.catalog.TableMeta;
import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.ipc.QueryMasterProtocol.*;
import org.apache.tajo.engine.exception.UnfinishedTaskException;
import org.apache.tajo.engine.json.CoreGsonHelper;
import org.apache.tajo.engine.planner.PlannerUtil;
@@ -45,14 +44,15 @@ import org.apache.tajo.engine.planner.logical.LogicalNode;
import org.apache.tajo.engine.planner.logical.SortNode;
import org.apache.tajo.engine.planner.logical.StoreTableNode;
import org.apache.tajo.engine.planner.physical.PhysicalExec;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService.Interface;
+import org.apache.tajo.ipc.TajoWorkerProtocol.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService.Interface;
import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
import org.apache.tajo.master.ExecutionBlock.PartitionType;
import org.apache.tajo.rpc.NullCallback;
import org.apache.tajo.storage.Fragment;
import org.apache.tajo.storage.StorageUtil;
import org.apache.tajo.storage.TupleComparator;
-import org.apache.tajo.worker.TaskRunner.WorkerContext;
+import org.apache.tajo.util.ApplicationIdUtils;
import java.io.File;
import java.io.IOException;
@@ -70,7 +70,7 @@ public class Task {
private final QueryConf conf;
private final FileSystem localFS;
- private final WorkerContext workerContext;
+ private final TaskRunner.TaskRunnerContext taskRunnerContext;
private final Interface masterProxy;
private final LocalDirAllocator lDirAllocator;
private final QueryUnitAttemptId taskId;
@@ -128,19 +128,19 @@ public class Task {
};
public Task(QueryUnitAttemptId taskId,
- final WorkerContext worker, final Interface masterProxy,
+ final TaskRunner.TaskRunnerContext worker, final Interface masterProxy,
final QueryUnitRequest request) throws IOException {
this.request = request;
this.reporter = new Reporter(masterProxy);
this.reporter.startCommunicationThread();
this.taskId = request.getId();
- this.conf = worker.getConf();
- this.workerContext = worker;
+ this.conf = worker.getQueryConf();
+ this.taskRunnerContext = worker;
this.masterProxy = masterProxy;
this.localFS = worker.getLocalFS();
this.lDirAllocator = worker.getLocalDirAllocator();
- this.taskDir = StorageUtil.concatPath(workerContext.getBaseDir(),
+ this.taskDir = StorageUtil.concatPath(taskRunnerContext.getBaseDir(),
taskId.getQueryUnitId().getId() + "_" + taskId.getId());
this.context = new TaskAttemptContext(conf, taskId,
@@ -162,7 +162,7 @@ public class Task {
// where ss is the subquery id associated with this task, and nnnnnn is the task id.
Path outFilePath = new Path(conf.getOutputPath(),
OUTPUT_FILE_PREFIX +
- OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskId.getSubQueryId().getId()) + "-" +
+ OUTPUT_FILE_FORMAT_SUBQUERY.get().format(taskId.getQueryUnitId().getExecutionBlockId().getId()) + "-" +
OUTPUT_FILE_FORMAT_TASK.get().format(taskId.getQueryUnitId().getId()));
LOG.info("Output File Path: " + outFilePath);
context.setOutputPath(outFilePath);
@@ -184,8 +184,10 @@ public class Task {
LOG.info("==> Table Id: " + f.getName() + ", url: " + f.getUrls());
}
LOG.info("* Local task dir: " + taskDir);
- LOG.info("* plan:\n");
- LOG.info(plan.toString());
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("* plan:\n");
+ LOG.debug(plan.toString());
+ }
LOG.info("==================================");
}
@@ -207,7 +209,6 @@ public class Task {
}
}
}
-
// for localizing the intermediate data
localize(request);
}
@@ -243,7 +244,7 @@ public class Task {
int i = fetcherRunners.size();
for (Fragment cache : cached) {
inFile = new Path(inputTableBaseDir, "in_" + i);
- workerContext.getDefaultFS().copyToLocalFile(cache.getPath(), inFile);
+ taskRunnerContext.getDefaultFS().copyToLocalFile(cache.getPath(), inFile);
cache.setPath(inFile);
i++;
}
@@ -273,7 +274,7 @@ public class Task {
public void fetch() {
for (Fetcher f : fetcherRunners) {
- workerContext.getFetchLauncher().submit(new FetchRunner(context, f));
+ taskRunnerContext.getFetchLauncher().submit(new FetchRunner(context, f));
}
}
@@ -298,8 +299,8 @@ public class Task {
// context.getWorkDir() 지우기
localFS.delete(context.getWorkDir(), true);
// tasks에서 자기 지우기
- synchronized (workerContext.getTasks()) {
- workerContext.getTasks().remove(this.getId());
+ synchronized (taskRunnerContext.getTasks()) {
+ taskRunnerContext.getTasks().remove(this.getId());
}
} catch (IOException e) {
e.printStackTrace();
@@ -312,7 +313,7 @@ public class Task {
public TaskStatusProto getReport() {
TaskStatusProto.Builder builder = TaskStatusProto.newBuilder();
- builder.setWorkerName(workerContext.getNodeId());
+ builder.setWorkerName(taskRunnerContext.getNodeId());
builder.setId(context.getTaskId().getProto())
.setProgress(context.getProgress()).setState(context.getState());
@@ -368,7 +369,7 @@ public class Task {
}
if (context.getFragmentSize() > 0) {
- this.executor = workerContext.getTQueryEngine().
+ this.executor = taskRunnerContext.getTQueryEngine().
createPlan(context, plan);
this.executor.init();
while(executor.next() != null && !killed) {
@@ -430,7 +431,7 @@ public class Task {
}
public void cleanupTask() {
- workerContext.getTasks().remove(getId());
+ taskRunnerContext.getTasks().remove(getId());
}
public int hashCode() {
@@ -623,7 +624,7 @@ public class Task {
String fileCache;
public String getFileCacheDir() {
fileCache = USERCACHE + "/" + "hyunsik" + "/" + APPCACHE + "/" +
- ConverterUtils.toString(taskId.getQueryId().getApplicationId()) +
+ ConverterUtils.toString(ApplicationIdUtils.queryIdToAppId(taskId.getQueryUnitId().getExecutionBlockId().getQueryId())) +
"/" + "output";
return fileCache;
}
@@ -631,7 +632,7 @@ public class Task {
public static Path getTaskAttemptDir(QueryUnitAttemptId quid) {
Path workDir =
StorageUtil.concatPath(
- quid.getSubQueryId().toString(),
+ quid.getQueryUnitId().getExecutionBlockId().toString(),
String.valueOf(quid.getQueryUnitId().getId()),
String.valueOf(quid.getId()));
return workDir;
[5/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
new file mode 100644
index 0000000..b2c129f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class QueryJobEvent extends AbstractEvent<QueryJobEvent.Type> {
+ private QueryInfo queryInfo;
+
+ public QueryJobEvent(Type type, QueryInfo queryInfo) {
+ super(type);
+
+ this.queryInfo = queryInfo;
+ }
+
+ public QueryInfo getQueryInfo() {
+ return this.queryInfo;
+ }
+
+ public enum Type {
+ QUERY_JOB_START,
+ QUERY_JOB_HEARTBEAT,
+ QUERY_JOB_FINISH,
+ QUERY_MASTER_START,
+ QUERY_MASTER_STOP
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
new file mode 100644
index 0000000..e4d83af
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.WorkerResource;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class QueryJobManager extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryJobManager.class.getName());
+
+ // TajoMaster Context
+ private final TajoMaster.MasterContext masterContext;
+
+ private AsyncDispatcher dispatcher;
+
+ private Map<QueryId, QueryInProgress> runningQueries = new HashMap<QueryId, QueryInProgress>();
+
+ private Map<QueryId, QueryInProgress> finishedQueries = new HashMap<QueryId, QueryInProgress>();
+
+ public QueryJobManager(final TajoMaster.MasterContext masterContext) {
+ super(QueryJobManager.class.getName());
+ this.masterContext = masterContext;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ try {
+ this.dispatcher = new AsyncDispatcher();
+ addService(this.dispatcher);
+
+ this.dispatcher.register(QueryJobEvent.Type.class, new QueryJobManagerEventHandler());
+ } catch (Exception e) {
+ catchException(null, e);
+ }
+
+ super.init(conf);
+ }
+
+ @Override
+ public void stop() {
+ synchronized(runningQueries) {
+ for(QueryInProgress eachQueryInProgress: runningQueries.values()) {
+ eachQueryInProgress.stop();
+ }
+ }
+ super.stop();
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ public EventHandler getEventHandler() {
+ return dispatcher.getEventHandler();
+ }
+
+ public QueryInfo createNewQueryJob(String sql, LogicalRootNode plan) throws Exception {
+ QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
+ QueryInProgress queryInProgress = new QueryInProgress(masterContext, queryId, sql, plan);
+
+ synchronized(runningQueries) {
+ runningQueries.put(queryId, queryInProgress);
+ }
+
+ addService(queryInProgress);
+ queryInProgress.init(getConfig());
+ queryInProgress.start();
+
+ queryInProgress.startQueryMaster();
+
+ return queryInProgress.getQueryInfo();
+ }
+
+ class QueryJobManagerEventHandler implements EventHandler<QueryJobEvent> {
+ @Override
+ public void handle(QueryJobEvent event) {
+ QueryInProgress queryInProgress = null;
+ synchronized(runningQueries) {
+ queryInProgress = runningQueries.get(event.getQueryInfo().getQueryId());
+ if(queryInProgress == null) {
+ LOG.warn("No query info in running queries.[" + event.getQueryInfo().getQueryId() + "]");
+ return;
+ }
+ }
+ queryInProgress.getEventHandler().handle(event);
+ }
+ }
+
+ public QueryInProgress getQueryInProgress(QueryId queryId) {
+ synchronized(runningQueries) {
+ return runningQueries.get(queryId);
+ }
+ }
+
+ public void stopQuery(QueryId queryId) {
+ LOG.info("====>Stop QueryInProgress:" + queryId);
+ QueryInProgress queryInProgress = getQueryInProgress(queryId);
+ if(queryInProgress != null) {
+ queryInProgress.stop();
+ synchronized(runningQueries) {
+ runningQueries.remove(queryId);
+ finishedQueries.put(queryId, queryInProgress);
+ }
+ } else {
+ LOG.warn("====> No QueryInProgress while query stopping: " + queryId);
+ }
+ }
+
+ private void catchException(QueryId queryId, Exception e) {
+ LOG.error(e.getMessage(), e);
+ QueryInProgress queryInProgress = runningQueries.get(queryId);
+ queryInProgress.catchException(e);
+ }
+
+ public synchronized TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand queryHeartbeat(
+ TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+ QueryInProgress queryInProgress = getQueryInProgress(new QueryId(queryHeartbeat.getQueryId()));
+ if(queryInProgress == null) {
+ return null;
+ }
+
+ QueryInfo queryInfo = makeQueryInfoFromHeartbeat(queryHeartbeat);
+ getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_HEARTBEAT, queryInfo));
+
+ return null;
+ }
+
+ private QueryInfo makeQueryInfoFromHeartbeat(TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+ QueryInfo queryInfo = new QueryInfo(new QueryId(queryHeartbeat.getQueryId()));
+ if(queryHeartbeat.getTajoWorkerHost() != null) {
+ WorkerResource queryMasterResource = new WorkerResource();
+ queryMasterResource.setAllocatedHost(queryHeartbeat.getTajoWorkerHost());
+ queryMasterResource.setPorts(new int[]{queryHeartbeat.getTajoWorkerPort(), queryHeartbeat.getTajoWorkerClientPort()});
+
+ queryInfo.setQueryMasterResource(queryMasterResource);
+ }
+ queryInfo.setLastMessage(queryHeartbeat.getStatusMessage());
+ queryInfo.setQueryState(queryHeartbeat.getState());
+
+ return queryInfo;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
index b66ef68..50ec5be 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -18,142 +18,82 @@
package org.apache.tajo.master.querymaster;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.SystemClock;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.client.YarnClient;
-import org.apache.hadoop.yarn.client.YarnClientImpl;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.proto.YarnProtos;
-import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.QueryConf;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
import org.apache.tajo.engine.planner.global.GlobalOptimizer;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.planner.logical.LogicalNode;
-import org.apache.tajo.engine.planner.logical.LogicalRootNode;
-import org.apache.tajo.engine.planner.logical.NodeType;
-import org.apache.tajo.engine.planner.logical.ScanNode;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.master.*;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.rm.RMContainerAllocator;
-import org.apache.tajo.rpc.ProtoAsyncRpcServer;
-import org.apache.tajo.rpc.ProtoBlockingRpcClient;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.GlobalPlanner;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.event.QueryStartEvent;
+import org.apache.tajo.rpc.NullCallback;
import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.worker.TajoWorker;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
+import java.util.ArrayList;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicBoolean;
-// TODO - when exception, send error status to QueryMasterManager
+// TODO - when exception, send error status to QueryJobManager
public class QueryMaster extends CompositeService implements EventHandler {
private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName());
- private static PrimitiveProtos.BoolProto TRUE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
- private static PrimitiveProtos.BoolProto FALSE_PROTO = PrimitiveProtos.BoolProto.newBuilder().setValue(false).build();
+
private static int QUERY_SESSION_TIMEOUT = 60 * 1000; //60 sec
- // AppMaster Common
- private final long appSubmitTime;
private Clock clock;
- // For Query
- private final QueryId queryId;
- private QueryContext queryContext;
- private Query query;
- private TajoProtos.QueryState state = TajoProtos.QueryState.QUERY_NOT_ASSIGNED;
- private String statusMessage;
- private MasterPlan masterPlan;
-
- private AsyncDispatcher dispatcher;
- private RMContainerAllocator rmAllocator;
+ private TajoAsyncDispatcher dispatcher;
- //service handler for QueryMasterManager, Worker
- private QueryMasterService queryMasterService;
- private QueryMasterClientService queryMasterClientService;
-
- private TaskRunnerLauncher taskRunnerLauncher;
private GlobalPlanner globalPlanner;
+
private GlobalOptimizer globalOptimizer;
- private boolean isCreateTableStmt;
+// private boolean isCreateTableStmt;
private StorageManager storageManager;
- private Path outputPath;
+
private QueryConf queryConf;
- private ApplicationAttemptId appAttemptId;
- private ApplicationId appId;
- private ProtoBlockingRpcClient queryMasterManagerClient;
- private QueryMasterManagerProtocol.QueryMasterManagerProtocolService.BlockingInterface queryMasterManagerService;
- private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+ private Map<QueryId, QueryMasterTask> queryMasterTasks = new HashMap<QueryId, QueryMasterTask>();
- private String queryMasterManagerAddress;
+ private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
- private YarnRPC yarnRPC;
+ private AtomicBoolean queryMasterStop = new AtomicBoolean(false);
- private YarnClient yarnClient;
+ private QueryMasterContext queryMasterContext;
- private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
+ private QueryHeartbeatThread queryHeartbeatThread;
- public QueryMaster(final QueryId queryId, final long appSubmitTime, String queryMasterManagerAddress) {
- super(QueryMaster.class.getName());
+ private TajoWorker.WorkerContext workerContext;
- this.queryId = queryId;
- this.appSubmitTime = appSubmitTime;
- this.appId = queryId.getApplicationId();
- this.queryMasterManagerAddress = queryMasterManagerAddress;
-
- LOG.info("Created Query Master for " + queryId);
+ public QueryMaster(TajoWorker.WorkerContext workerContext) {
+ super(QueryMaster.class.getName());
+ this.workerContext = workerContext;
}
public void init(Configuration conf) {
+ LOG.info("QueryMaster init");
try {
queryConf = new QueryConf(conf);
- QUERY_SESSION_TIMEOUT = 60 * 1000;//queryConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
- queryContext = new QueryContext(queryConf);
- yarnRPC = YarnRPC.create(queryContext.getConf());
- connectYarnClient();
+ queryConf.addResource(new Path(QueryConf.QUERY_MASTER_FILENAME));
- LOG.info("Init QueryMasterManagerClient connection to:" + queryMasterManagerAddress);
- InetSocketAddress addr = NetUtils.createSocketAddr(queryMasterManagerAddress);
- queryMasterManagerClient = new ProtoBlockingRpcClient(QueryMasterManagerProtocol.class, addr);
- queryMasterManagerService = queryMasterManagerClient.getStub();
+ QUERY_SESSION_TIMEOUT = 60 * 1000;//queryConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
+ queryMasterContext = new QueryMasterContext(queryConf);
clock = new SystemClock();
- this.dispatcher = new AsyncDispatcher();
+ this.dispatcher = new TajoAsyncDispatcher("querymaster_" + System.currentTimeMillis());
addIfService(dispatcher);
this.storageManager = new StorageManager(queryConf);
@@ -161,23 +101,8 @@ public class QueryMaster extends CompositeService implements EventHandler {
globalPlanner = new GlobalPlanner(queryConf, storageManager, dispatcher.getEventHandler());
globalOptimizer = new GlobalOptimizer();
- queryMasterService = new QueryMasterService();
- addIfService(queryMasterService);
-
- queryMasterClientService = new QueryMasterClientService(queryContext);
- addIfService(queryMasterClientService);
-
- initStagingDir();
-
- dispatcher.register(SubQueryEventType.class, new SubQueryEventDispatcher());
- dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
- dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
- dispatcher.register(QueryFinishEvent.EventType.class, new QueryFinishEventHandler());
- dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+ dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
- clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
-
- clientSessionTimeoutCheckThread.start();
} catch (Throwable t) {
LOG.error(t.getMessage(), t);
throw new RuntimeException(t);
@@ -185,364 +110,88 @@ public class QueryMaster extends CompositeService implements EventHandler {
super.init(conf);
}
- class ClientSessionTimeoutCheckThread extends Thread {
- public void run() {
- LOG.info("ClientSessionTimeoutCheckThread started");
- while(true) {
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e) {
- break;
- }
- try {
- long lastHeartbeat = queryContext.getLastClientHeartbeat();
- long time = System.currentTimeMillis() - lastHeartbeat;
- if(lastHeartbeat > 0 && time > QUERY_SESSION_TIMEOUT) {
- LOG.warn("Query " + queryId + " stopped cause query sesstion timeout: " + time + " ms");
- QueryMaster.this.stop();
- }
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- }
- }
- }
-
- class QueryHeartbeatThread extends Thread {
- public QueryHeartbeatThread() {
- super("QueryHeartbeatThread");
- }
-
- @Override
- public void run() {
- LOG.info("Start QueryMaster heartbeat thread");
- while(queryMasterManagerClient.isConnected()) {
- QueryMasterManagerProtocol.QueryHeartbeat queryHeartbeat =
- QueryMasterManagerProtocol.QueryHeartbeat.newBuilder()
- .setQueryMasterHost(queryMasterService.bindAddr.getHostName())
- .setQueryMasterPort(queryMasterService.bindAddr.getPort())
- .setQueryMasterClientPort(queryMasterClientService.getBindAddr().getPort())
- .setState(state)
- .setQueryId(queryId.getProto())
- .build();
-
- try {
- QueryMasterManagerProtocol.QueryHeartbeatResponse response =
- queryMasterManagerService.queryHeartbeat(null, queryHeartbeat);
- if(response.getResponseCommand() != null) {
- if("executeQuery".equals(response.getResponseCommand().getCommand())) {
- appAttemptId = TajoIdUtils.toApplicationAttemptId(response.getResponseCommand().getParams(0));
- startQuery(response.getResponseCommand().getParams(1),
- response.getResponseCommand().getParams(2));
- }
- }
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- try {
- Thread.sleep(2000);
- } catch (InterruptedException e) {
- break;
- }
- }
- LOG.info("QueryMaster heartbeat thread stopped");
- }
- }
-
- // TODO blocking/nonblocking ???
- class QueryMasterService extends AbstractService implements QueryMasterProtocol.QueryMasterProtocolService.Interface {
- private ProtoAsyncRpcServer rpcServer;
- private InetSocketAddress bindAddr;
- private String addr;
- private QueryHeartbeatThread queryHeartbeatThread;
-
- public QueryMasterService() {
- super(QueryMasterService.class.getName());
-
- // Setup RPC server
- try {
- InetSocketAddress initIsa =
- new InetSocketAddress(InetAddress.getLocalHost(), 0);
- if (initIsa.getAddress() == null) {
- throw new IllegalArgumentException("Failed resolve of " + initIsa);
- }
-
- this.rpcServer = new ProtoAsyncRpcServer(QueryMasterProtocol.class, this, initIsa);
- this.rpcServer.start();
-
- this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
- this.addr = NetUtils.normalizeInetSocketAddress(this.bindAddr);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- queryConf.setVar(TajoConf.ConfVars.TASKRUNNER_LISTENER_ADDRESS, addr);
- LOG.info("QueryMasterService startup");
- }
-
- @Override
- public void init(Configuration conf) {
- super.init(conf);
- }
-
- @Override
- public void start() {
- try {
- queryHeartbeatThread = new QueryHeartbeatThread();
- queryHeartbeatThread.start();
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- // TODO - set query status failed and stop QueryMaster
- }
- super.start();
- }
-
- @Override
- public void stop() {
- if(rpcServer != null) {
- rpcServer.shutdown();
- }
- if(queryHeartbeatThread != null) {
- queryHeartbeatThread.interrupt();
- }
- if(yarnClient != null) {
- yarnClient.stop();
- }
- if(clientSessionTimeoutCheckThread != null) {
- clientSessionTimeoutCheckThread.interrupt();
- }
- super.stop();
- LOG.info("QueryMasterService stopped");
- }
-
- @Override
- public void getTask(RpcController controller, YarnProtos.ContainerIdProto request,
- RpcCallback<QueryMasterProtocol.QueryUnitRequestProto> done) {
- queryContext.getEventHandler().handle(new TaskRequestEvent(new ContainerIdPBImpl(request), done));
- }
-
- @Override
- public void statusUpdate(RpcController controller, QueryMasterProtocol.TaskStatusProto request,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- QueryUnitAttemptId attemptId = new QueryUnitAttemptId(request.getId());
- queryContext.getEventHandler().handle(new TaskAttemptStatusUpdateEvent(attemptId, request));
- done.run(TRUE_PROTO);
- }
-
- @Override
- public void ping(RpcController controller,
- TajoIdProtos.QueryUnitAttemptIdProto attemptIdProto,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- // TODO - to be completed
- QueryUnitAttemptId attemptId = new QueryUnitAttemptId(attemptIdProto);
- done.run(TRUE_PROTO);
- }
-
- @Override
- public void fatalError(RpcController controller, QueryMasterProtocol.TaskFatalErrorReport report,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- queryContext.getEventHandler().handle(new TaskFatalErrorEvent(report));
- done.run(TRUE_PROTO);
- }
+ @Override
+ public void start() {
+ LOG.info("====>QueryMaster start");
- @Override
- public void done(RpcController controller, QueryMasterProtocol.TaskCompletionReport report,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- queryContext.getEventHandler().handle(new TaskCompletionEvent(report));
- done.run(TRUE_PROTO);
- }
+ queryHeartbeatThread = new QueryHeartbeatThread();
+ queryHeartbeatThread.start();
- @Override
- public void executeQuery(RpcController controller, PrimitiveProtos.StringProto request,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- }
- }
+ clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
+ clientSessionTimeoutCheckThread.start();
- public void start() {
super.start();
}
+ @Override
public void stop() {
- LOG.info("unregisterApplicationMaster");
- if(rmAllocator != null) {
- try {
- FinalApplicationStatus status = FinalApplicationStatus.UNDEFINED;
- if (query != null) {
- TajoProtos.QueryState state = query.getState();
- if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
- status = FinalApplicationStatus.SUCCEEDED;
- } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
- status = FinalApplicationStatus.FAILED;
- } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
- status = FinalApplicationStatus.FAILED;
- }
- }
- this.rmAllocator.unregisterApplicationMaster(status, "tajo query finished", null);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
+ synchronized(queryMasterStop) {
+ if(queryMasterStop.get()) {
+ return;
}
- }
- // TODO - release opened resource
- if(this.queryMasterManagerClient != null) {
- reportQueryStatus();
-
- queryMasterManagerClient.close();
+ queryMasterStop.set(true);
+ queryMasterStop.notifyAll();
}
- try {
- FileSystem.closeAll();
- } catch (IOException e) {
- LOG.error(e.getMessage(), e);
+ if(queryHeartbeatThread != null) {
+ queryHeartbeatThread.interrupt();
}
+ if(clientSessionTimeoutCheckThread != null) {
+ clientSessionTimeoutCheckThread.interrupt();
+ }
super.stop();
- synchronized(queryId) {
- queryId.notifyAll();
+ LOG.info("QueryMaster stop");
+ if(!queryMasterContext.getWorkerContext().isStandbyMode()) {
+ queryMasterContext.getWorkerContext().stopWorker(true);
}
}
- private void reportQueryStatus() {
- //send query status heartbeat
- QueryMasterManagerProtocol.QueryHeartbeat.Builder queryHeartbeatBuilder =
- QueryMasterManagerProtocol.QueryHeartbeat.newBuilder()
- .setQueryMasterHost(queryMasterService.bindAddr.getHostName())
- .setQueryMasterPort(queryMasterService.bindAddr.getPort())
- .setQueryMasterClientPort(queryMasterClientService.getBindAddr().getPort())
- .setState(state)
- .setQueryId(queryId.getProto());
-
- if(statusMessage != null) {
- queryHeartbeatBuilder.setStatusMessage(statusMessage);
- }
+ public void reportQueryStatusToQueryMaster(QueryId queryId, TajoProtos.QueryState state) {
+ LOG.info("Send QueryMaster Ready to QueryJobManager:" + queryId);
try {
- queryMasterManagerService.queryHeartbeat(null, queryHeartbeatBuilder.build());
+ TajoMasterProtocol.TajoHeartbeat.Builder queryHeartbeatBuilder = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+ .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+ .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+ .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+ .setState(state)
+ .setQueryId(queryId.getProto());
+
+ workerContext.getTajoMasterRpcClient().heartbeat(null, queryHeartbeatBuilder.build(), NullCallback.get());
} catch (Exception e) {
LOG.error(e.getMessage(), e);
}
}
- private void connectYarnClient() {
- this.yarnClient = new YarnClientImpl();
- this.yarnClient.init(queryConf);
- this.yarnClient.start();
- }
-
protected void addIfService(Object object) {
if (object instanceof Service) {
addService((Service) object);
}
}
- public synchronized void startQuery(String queryStr, String planJSON) {
- LOG.info("Query Start:" + queryStr);
- LOG.info("Plan JSON:" + planJSON);
- if(query != null) {
- LOG.warn("Query already started");
- return;
- }
-
- try {
- LogicalRootNode logicalNodeRoot = (LogicalRootNode) CoreGsonHelper.fromJson(planJSON, LogicalNode.class);
- LogicalNode[] scanNodes = PlannerUtil.findAllNodes(logicalNodeRoot, NodeType.SCAN);
- if(scanNodes != null) {
- for(LogicalNode eachScanNode: scanNodes) {
- ScanNode scanNode = (ScanNode)eachScanNode;
- tableDescMap.put(scanNode.getFromTable().getTableName(), scanNode.getFromTable().getTableDesc());
- }
- }
- MasterPlan globalPlan = globalPlanner.build(queryId, logicalNodeRoot);
- this.masterPlan = globalOptimizer.optimize(globalPlan);
-
- taskRunnerLauncher = new TaskRunnerLauncherImpl(queryContext);
- addIfService(taskRunnerLauncher);
- dispatcher.register(TaskRunnerGroupEvent.EventType.class, taskRunnerLauncher);
-
- ((TaskRunnerLauncherImpl)taskRunnerLauncher).init(queryConf);
- ((TaskRunnerLauncherImpl)taskRunnerLauncher).start();
-
- rmAllocator = new RMContainerAllocator(queryContext);
- addIfService(rmAllocator);
- dispatcher.register(ContainerAllocatorEventType.class, rmAllocator);
-
- rmAllocator.init(queryConf);
- rmAllocator.start();
-
- //TODO - synchronized with executeQuery logic
- query = new Query(queryContext, queryId, clock, appSubmitTime,
- "", dispatcher.getEventHandler(), masterPlan, storageManager);
- dispatcher.register(QueryEventType.class, query);
-
- dispatcher.getEventHandler().handle(new QueryEvent(queryId,
- QueryEventType.INIT));
- dispatcher.getEventHandler().handle(new QueryEvent(queryId,
- QueryEventType.START));
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- //send FAIL query status
- this.statusMessage = StringUtils.stringifyException(e);
- this.state = TajoProtos.QueryState.QUERY_FAILED;
- }
- }
-
@Override
public void handle(Event event) {
dispatcher.getEventHandler().handle(event);
}
- public EventHandler getEventHandler() {
- return dispatcher.getEventHandler();
- }
-
- private class SubQueryEventDispatcher implements EventHandler<SubQueryEvent> {
- public void handle(SubQueryEvent event) {
- SubQueryId id = event.getSubQueryId();
- query.getSubQuery(id).handle(event);
- }
+ public Query getQuery(QueryId queryId) {
+ return queryMasterTasks.get(queryId).getQuery();
}
- private class TaskEventDispatcher
- implements EventHandler<TaskEvent> {
- public void handle(TaskEvent event) {
- QueryUnitId taskId = event.getTaskId();
- QueryUnit task = query.getSubQuery(taskId.getSubQueryId()).
- getQueryUnit(taskId);
- task.handle(event);
- }
- }
-
- private class TaskAttemptEventDispatcher
- implements EventHandler<TaskAttemptEvent> {
- public void handle(TaskAttemptEvent event) {
- QueryUnitAttemptId attemptId = event.getTaskAttemptId();
- SubQuery subQuery = query.getSubQuery(attemptId.getSubQueryId());
- QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
- QueryUnitAttempt attempt = task.getAttempt(attemptId);
- attempt.handle(event);
- }
- }
-
- private class TaskSchedulerDispatcher
- implements EventHandler<TaskSchedulerEvent> {
- public void handle(TaskSchedulerEvent event) {
- SubQuery subQuery = query.getSubQuery(event.getSubQueryId());
- subQuery.getTaskScheduler().handle(event);
- }
+ public QueryMasterTask getQueryMasterTask(QueryId queryId) {
+ return queryMasterTasks.get(queryId);
}
- public QueryContext getContext() {
- return this.queryContext;
+ public QueryMasterContext getContext() {
+ return this.queryMasterContext;
}
- public class QueryContext {
+ public class QueryMasterContext {
private QueryConf conf;
- public Map<ContainerId, ContainerProxy> containers = new ConcurrentHashMap<ContainerId, ContainerProxy>();
- int minCapability;
- int maxCapability;
- int numCluster;
- AtomicLong lastClientHeartbeat = new AtomicLong(-1);
- public QueryContext(QueryConf conf) {
+ public QueryMasterContext(QueryConf conf) {
this.conf = conf;
}
@@ -550,15 +199,7 @@ public class QueryMaster extends CompositeService implements EventHandler {
return conf;
}
- public InetSocketAddress getQueryMasterServiceAddress() {
- return queryMasterService.bindAddr;
- }
-
- public QueryMasterClientService getQueryMasterClientService() {
- return queryMasterClientService;
- }
-
- public AsyncDispatcher getDispatcher() {
+ public TajoAsyncDispatcher getDispatcher() {
return dispatcher;
}
@@ -566,251 +207,133 @@ public class QueryMaster extends CompositeService implements EventHandler {
return clock;
}
- public Query getQuery() {
- return query;
+ public StorageManager getStorageManager() {
+ return storageManager;
}
- public SubQuery getSubQuery(SubQueryId subQueryId) {
- return query.getSubQuery(subQueryId);
+ public QueryMaster getQueryMaster() {
+ return QueryMaster.this;
}
- public QueryId getQueryId() {
- return queryId;
+ public GlobalPlanner getGlobalPlanner() {
+ return globalPlanner;
}
-
- public ApplicationId getApplicationId() {
- return appId;
+ public GlobalOptimizer getGlobalOptimizer() {
+ return globalOptimizer;
}
- public ApplicationAttemptId getApplicationAttemptId() {
- return appAttemptId;
+ public TajoWorker.WorkerContext getWorkerContext() {
+ return workerContext;
}
public EventHandler getEventHandler() {
return dispatcher.getEventHandler();
}
- public void addContainer(ContainerId cId, ContainerProxy container) {
- containers.put(cId, container);
- }
-
- public void removeContainer(ContainerId cId) {
- containers.remove(cId);
- }
-
- public boolean containsContainer(ContainerId cId) {
- return containers.containsKey(cId);
- }
-
- public ContainerProxy getContainer(ContainerId cId) {
- return containers.get(cId);
- }
-
- public Map<ContainerId, ContainerProxy> getContainers() {
- return containers;
- }
-
- public int getNumClusterNode() {
- return numCluster;
- }
-
- public void setNumClusterNodes(int num) {
- numCluster = num;
- }
-
-// public CatalogService getCatalog() {
-// return catalog;
-// }
-
- public Map<String, TableDesc> getTableDescMap() {
- return tableDescMap;
- }
-
- public Path getOutputPath() {
- return outputPath;
- }
-
- public void setMaxContainerCapability(int capability) {
- this.maxCapability = capability;
- }
-
- public int getMaxContainerCapability() {
- return this.maxCapability;
- }
-
- public void setMinContainerCapability(int capability) {
- this.minCapability = capability;
- }
-
- public int getMinContainerCapability() {
- return this.minCapability;
- }
-
- public boolean isCreateTableQuery() {
- return isCreateTableStmt;
- }
-
- public float getProgress() {
- if(query != null) {
- return query.getProgress();
- } else {
- return 0;
+ public void stopQuery(QueryId queryId) {
+ QueryMasterTask queryMasterTask;
+ synchronized(queryMasterTasks) {
+ queryMasterTask = queryMasterTasks.remove(queryId);
}
- }
-
- public long getStartTime() {
- if(query != null) {
- return query.getStartTime();
+ if(queryMasterTask != null) {
+ try {
+ queryMasterTask.stop();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
} else {
- return -1;
+ LOG.warn("No query info:" + queryId);
}
- }
-
- public long getFinishTime() {
- if(query != null) {
- return query.getFinishTime();
- } else {
- return -1;
+ if(!workerContext.isStandbyMode()) {
+ stop();
}
}
-
- public StorageManager getStorageManager() {
- return storageManager;
- }
-
- public QueryMaster getQueryMaster() {
- return QueryMaster.this;
- }
-
- public YarnRPC getYarnRPC() {
- return yarnRPC;
- }
-
- public void setState(TajoProtos.QueryState state) {
- QueryMaster.this.state = state;
- }
-
- public TajoProtos.QueryState getState() {
- return state;
- }
-
- public void touchSessionTime() {
- this.lastClientHeartbeat.set(System.currentTimeMillis());
- }
-
- public long getLastClientHeartbeat() {
- return this.lastClientHeartbeat.get();
- }
}
- private class QueryFinishEventHandler implements EventHandler<QueryFinishEvent> {
+ private class QueryStartEventHandler implements EventHandler<QueryStartEvent> {
@Override
- public void handle(QueryFinishEvent event) {
- LOG.info("Query end notification started for QueryId : " + query.getId() + "," + query.getState());
-
- //QueryMaster must be lived until client fetching all query result data.
- try {
- // Stop all services
- // This will also send the final report to the ResourceManager
- //LOG.info("Calling stop for all the services");
-// stop();
- } catch (Throwable t) {
- LOG.warn("Graceful stop failed ", t);
+ public void handle(QueryStartEvent event) {
+ LOG.info("====>Start QueryStartEventHandler:" + event.getQueryId());
+ //To change body of implemented methods use File | Settings | File Templates.
+ QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
+ event.getQueryId(), event.getLogicalPlanJson());
+
+ queryMasterTask.init(queryConf);
+ queryMasterTask.start();
+ synchronized(queryMasterTasks) {
+ queryMasterTasks.put(event.getQueryId(), queryMasterTask);
}
-
- //Bring the process down by force.
- //Not needed after HADOOP-7140
- //LOG.info("Exiting QueryMaster..GoodBye!");
}
}
- // query submission directory is private!
- final public static FsPermission USER_DIR_PERMISSION =
- FsPermission.createImmutable((short) 0700); // rwx--------
-
- /**
- * It initializes the final output and staging directory and sets
- * them to variables.
- */
- private void initStagingDir() throws IOException {
- QueryConf conf = getContext().getConf();
-
- String realUser;
- String currentUser;
- UserGroupInformation ugi;
- ugi = UserGroupInformation.getLoginUser();
- realUser = ugi.getShortUserName();
- currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
-
- String givenOutputTableName = conf.getOutputTable();
- Path stagingDir;
-
- // If final output directory is not given by an user,
- // we use the query id as a output directory.
- if (givenOutputTableName.equals("")) {
- this.isCreateTableStmt = false;
- FileSystem defaultFS = FileSystem.get(conf);
-
- Path homeDirectory = defaultFS.getHomeDirectory();
- if (!defaultFS.exists(homeDirectory)) {
- defaultFS.mkdirs(homeDirectory, new FsPermission(USER_DIR_PERMISSION));
- }
-
- Path userQueryDir = new Path(homeDirectory, TajoConstants.USER_QUERYDIR_PREFIX);
-
- if (defaultFS.exists(userQueryDir)) {
- FileStatus fsStatus = defaultFS.getFileStatus(userQueryDir);
- String owner = fsStatus.getOwner();
+ class QueryHeartbeatThread extends Thread {
+ public QueryHeartbeatThread() {
+ super("QueryHeartbeatThread");
+ }
- if (!(owner.equals(currentUser) || owner.equals(realUser))) {
- throw new IOException("The ownership on the user's query " +
- "directory " + userQueryDir + " is not as expected. " +
- "It is owned by " + owner + ". The directory must " +
- "be owned by the submitter " + currentUser + " or " +
- "by " + realUser);
+ @Override
+ public void run() {
+ LOG.info("Start QueryMaster heartbeat thread");
+ while(!queryMasterStop.get()) {
+ //TODO report all query status
+ List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+ synchronized(queryMasterTasks) {
+ tempTasks.addAll(queryMasterTasks.values());
}
+ synchronized(queryMasterTasks) {
+ for(QueryMasterTask eachTask: tempTasks) {
+ TajoMasterProtocol.TajoHeartbeat queryHeartbeat = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+ .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+ .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+ .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+ .setState(eachTask.getState())
+ .setQueryId(eachTask.getQueryId().getProto())
+ .build();
- if (!fsStatus.getPermission().equals(USER_DIR_PERMISSION)) {
- LOG.info("Permissions on staging directory " + userQueryDir + " are " +
- "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
- "to correct value " + USER_DIR_PERMISSION);
- defaultFS.setPermission(userQueryDir, new FsPermission(USER_DIR_PERMISSION));
+ workerContext.getTajoMasterRpcClient().heartbeat(null, queryHeartbeat, NullCallback.get());
+ }
+ }
+ synchronized(queryMasterStop) {
+ try {
+ queryMasterStop.wait(2000);
+ } catch (InterruptedException e) {
+ break;
+ }
}
- } else {
- defaultFS.mkdirs(userQueryDir,
- new FsPermission(USER_DIR_PERMISSION));
- }
-
- stagingDir = StorageUtil.concatPath(userQueryDir, queryId.toString());
-
- if (defaultFS.exists(stagingDir)) {
- throw new IOException("The staging directory " + stagingDir
- + "already exists. The directory must be unique to each query");
- } else {
- defaultFS.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
}
+ LOG.info("QueryMaster heartbeat thread stopped");
+ }
+ }
- // Set the query id to the output table name
- conf.setOutputTable(queryId.toString());
- } else {
- this.isCreateTableStmt = true;
- Path warehouseDir = new Path(conf.getVar(TajoConf.ConfVars.ROOT_DIR),
- TajoConstants.WAREHOUSE_DIR);
- stagingDir = new Path(warehouseDir, conf.getOutputTable());
+ class ClientSessionTimeoutCheckThread extends Thread {
+ public void run() {
+ LOG.info("ClientSessionTimeoutCheckThread started");
+ while(true) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ break;
+ }
+ List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+ synchronized(queryMasterTasks) {
+ tempTasks.addAll(queryMasterTasks.values());
+ }
- FileSystem fs = warehouseDir.getFileSystem(conf);
- if (fs.exists(stagingDir)) {
- throw new IOException("The staging directory " + stagingDir
- + " already exists. The directory must be unique to each query");
- } else {
- // TODO - should have appropriate permission
- fs.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
+ for(QueryMasterTask eachTask: tempTasks) {
+ try {
+ long lastHeartbeat = eachTask.getLastClientHeartbeat();
+ long time = System.currentTimeMillis() - lastHeartbeat;
+ if(lastHeartbeat > 0 && time > QUERY_SESSION_TIMEOUT) {
+ LOG.warn("Query " + eachTask.getQueryId() + " stopped cause query sesstion timeout: " + time + " ms");
+ eachTask.expiredSessionTimeout();
+ }
+ } catch (Exception e) {
+ LOG.error(eachTask.getQueryId() + ":" + e.getMessage(), e);
+ }
+ }
}
}
-
- conf.setOutputPath(stagingDir);
- outputPath = stagingDir;
- LOG.info("Initialized Query Staging Dir: " + outputPath);
}
+
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
deleted file mode 100644
index 74298e5..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterClientService.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.proto.YarnProtos;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.ipc.ClientProtos;
-import org.apache.tajo.ipc.QueryMasterClientProtocol;
-import org.apache.tajo.rpc.ProtoBlockingRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-
-public class QueryMasterClientService extends AbstractService {
- private static final Log LOG = LogFactory.getLog(QueryMasterClientService.class);
- private final PrimitiveProtos.BoolProto BOOL_TRUE =
- PrimitiveProtos.BoolProto.newBuilder().setValue(true).build();
-
- private ProtoBlockingRpcServer rpcServer;
- private InetSocketAddress bindAddr;
- private String addr;
- private QueryMaster.QueryContext queryContext;
- private QueryMasterClientProtocolServiceHandler serviceHandler;
-
- public QueryMasterClientService(QueryMaster.QueryContext queryContext) {
- super(QueryMasterClientService.class.getName());
-
- this.queryContext = queryContext;
- this.serviceHandler = new QueryMasterClientProtocolServiceHandler();
-
- // init RPC Server in constructor cause Heartbeat Thread use bindAddr
- // Setup RPC server
- try {
- // TODO initial port num is value of config and find unused port with sequence
- InetSocketAddress initIsa = new InetSocketAddress(InetAddress.getLocalHost(), 0);
- if (initIsa.getAddress() == null) {
- throw new IllegalArgumentException("Failed resolve of " + initIsa);
- }
-
- // TODO blocking/non-blocking??
- this.rpcServer = new ProtoBlockingRpcServer(QueryMasterClientProtocol.class, serviceHandler, initIsa);
- this.rpcServer.start();
-
- this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
- this.addr = NetUtils.normalizeInetSocketAddress(bindAddr);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- // Get the master address
- LOG.info(QueryMasterClientService.class.getSimpleName() + " (" + queryContext.getQueryId() + ") listens on "
- + addr);
- }
-
- @Override
- public void init(Configuration conf) {
- super.init(conf);
- }
-
- @Override
- public void start() {
- super.start();
- }
-
- @Override
- public void stop() {
- if(rpcServer != null) {
- rpcServer.shutdown();
- }
- LOG.info("QueryMasterClientService stopped");
- super.stop();
- }
-
- public InetSocketAddress getBindAddr() {
- return bindAddr;
- }
-
-
- public class QueryMasterClientProtocolServiceHandler
- implements QueryMasterClientProtocol.QueryMasterClientProtocolService.BlockingInterface {
- @Override
- public PrimitiveProtos.BoolProto updateSessionVariables(
- RpcController controller,
- ClientProtos.UpdateSessionVariableRequest request) throws ServiceException {
- return null;
- }
-
- @Override
- public ClientProtos.GetQueryResultResponse getQueryResult(
- RpcController controller,
- ClientProtos.GetQueryResultRequest request) throws ServiceException {
- QueryId queryId = new QueryId(request.getQueryId());
- Query query = queryContext.getQuery();
-
- ClientProtos.GetQueryResultResponse.Builder builder = ClientProtos.GetQueryResultResponse.newBuilder();
-
- if(query == null) {
- builder.setErrorMessage("No Query for " + queryId);
- } else {
- switch (query.getState()) {
- case QUERY_SUCCEEDED:
- builder.setTableDesc((CatalogProtos.TableDescProto)query.getResultDesc().getProto());
- break;
- case QUERY_FAILED:
- case QUERY_ERROR:
- builder.setErrorMessage("Query " + queryId + " is failed");
- default:
- builder.setErrorMessage("Query " + queryId + " is still running");
- }
- }
- return builder.build();
- }
-
- @Override
- public ClientProtos.GetQueryStatusResponse getQueryStatus(
- RpcController controller,
- ClientProtos.GetQueryStatusRequest request) throws ServiceException {
- ClientProtos.GetQueryStatusResponse.Builder builder
- = ClientProtos.GetQueryStatusResponse.newBuilder();
- QueryId queryId = new QueryId(request.getQueryId());
- builder.setQueryId(request.getQueryId());
-
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
- builder.setResultCode(ClientProtos.ResultCode.OK);
- builder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
- } else {
- Query query = queryContext.getQuery();
- builder.setResultCode(ClientProtos.ResultCode.OK);
- builder.setQueryMasterHost(queryContext.getQueryMasterClientService().getBindAddr().getHostName());
- builder.setQueryMasterPort(queryContext.getQueryMasterClientService().getBindAddr().getPort());
-
-
- queryContext.touchSessionTime();
- if (query != null) {
- builder.setState(query.getState());
- builder.setProgress(query.getProgress());
- builder.setSubmitTime(query.getAppSubmitTime());
- builder.setInitTime(query.getInitializationTime());
- builder.setHasResult(!query.isCreateTableStmt());
- if (query.getState() == TajoProtos.QueryState.QUERY_SUCCEEDED) {
- builder.setFinishTime(query.getFinishTime());
- } else {
- builder.setFinishTime(System.currentTimeMillis());
- }
- } else {
- builder.setState(queryContext.getState());
- }
- }
-
- return builder.build();
- }
-
- @Override
- public PrimitiveProtos.BoolProto killQuery(
- RpcController controller,
- YarnProtos.ApplicationAttemptIdProto request) throws ServiceException {
- LOG.info("Stop QueryMaster:" + queryContext.getQueryId());
- Thread t = new Thread() {
- public void run() {
- try {
- Thread.sleep(1000); //wait tile return to rpc response
- } catch (InterruptedException e) {
- }
- queryContext.getQueryMaster().stop();
- }
- };
- t.start();
- return BOOL_TRUE;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
deleted file mode 100644
index 35d7201..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManager.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.Clock;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.client.YarnClient;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.service.CompositeService;
-import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.engine.planner.logical.LogicalRootNode;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol.QueryHeartbeatResponse;
-import org.apache.tajo.master.ContainerProxy;
-import org.apache.tajo.master.TajoMaster;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-// TODO - check QueryMaster status and if QueryMaster failed, release resource
-public class QueryMasterManager extends CompositeService {
- private static final Log LOG = LogFactory.getLog(QueryMasterManager.class.getName());
-
- // Master Context
- private final TajoMaster.MasterContext masterContext;
-
- // AppMaster Common
- private final Clock clock;
- private final long appSubmitTime;
- private final ApplicationId appId;
- private ApplicationAttemptId appAttemptId;
-
- protected YarnClient yarnClient;
-
- // For Query
- private final QueryId queryId;
-
- private AsyncDispatcher dispatcher;
- private YarnRPC rpc;
-
- private TajoProtos.QueryState state;
- private float progress;
- private long finishTime;
- private TableDesc resultDesc;
- private String queryMasterHost;
- private int queryMasterPort;
- private int queryMasterClientPort;
-
- private LogicalRootNode plan;
-
- private AtomicBoolean querySubmitted = new AtomicBoolean(false);
-
- private AtomicBoolean queryMasterStopped = new AtomicBoolean(true);
-
- private boolean stopCheckThreadStarted = false;
-
- private String query;
-
- public QueryMasterManager(final TajoMaster.MasterContext masterContext,
- final YarnClient yarnClient,
- final QueryId queryId,
- final String query,
- final LogicalRootNode plan,
- final ApplicationId appId,
- final Clock clock, long appSubmitTime) {
- super(QueryMasterManager.class.getName());
- this.masterContext = masterContext;
- this.yarnClient = yarnClient;
-
- this.appId = appId;
- this.clock = clock;
- this.appSubmitTime = appSubmitTime;
- this.queryId = queryId;
- this.plan = plan;
- this.query = query;
- LOG.info("Created Query Master Manager for AppId=" + appId + ", QueryID=" + queryId);
- }
-
- @Override
- public void init(Configuration conf) {
- super.init(conf);
-
- state = TajoProtos.QueryState.QUERY_MASTER_INIT;
- }
-
- public TajoProtos.QueryState getState() {
- return state;
- }
-
- @Override
- public void start() {
- try {
- appAttemptId = allocateAndLaunchQueryMaster();
- } catch (YarnRemoteException e) {
- LOG.error(e.getMessage(), e);
- }
- super.start();
- }
-
- @Override
- public void stop() {
- while(true) {
- if(queryMasterStopped.get()) {
- break;
- }
- try {
- Thread.sleep(100);
- } catch (InterruptedException e) {
- }
- }
- LOG.info("QueryMasterManager for " + queryId + " stopped");
- super.stop();
- }
-
- public float getProgress() {
- return progress;
- }
-
- public long getAppSubmitTime() {
- return appSubmitTime;
- }
-
- public long getFinishTime() {
- return finishTime;
- }
-
- public TableDesc getResultDesc() {
- return resultDesc;
- }
-
- public String getQueryMasterHost() {
- return queryMasterHost;
- }
-
- public int getQueryMasterPort() {
- return queryMasterPort;
- }
-
- public int getQueryMasterClientPort() {
- return queryMasterClientPort;
- }
-
- public synchronized QueryHeartbeatResponse.ResponseCommand queryHeartbeat(QueryMasterManagerProtocol.QueryHeartbeat queryHeartbeat) {
- this.queryMasterHost = queryHeartbeat.getQueryMasterHost();
- this.queryMasterPort = queryHeartbeat.getQueryMasterPort();
- this.queryMasterClientPort = queryHeartbeat.getQueryMasterClientPort();
- this.state = queryHeartbeat.getState();
- if(state == TajoProtos.QueryState.QUERY_FAILED) {
- //TODO needed QueryMaster's detail status(failed before or after launching worker)
- queryMasterStopped.set(true);
- if(queryHeartbeat.getStatusMessage() != null) {
- LOG.warn(queryId + " failed, " + queryHeartbeat.getStatusMessage());
- }
- }
-
- if(!stopCheckThreadStarted && !queryMasterStopped.get() && isFinishState(this.state)) {
- stopCheckThreadStarted = true;
- startCheckingQueryMasterStop();
- }
- if(appAttemptId != null && !querySubmitted.get()) {
- LOG.info("submitQuery to QueryMaster(" + queryMasterHost + ":" + queryMasterPort + ")");
- queryMasterStopped.set(false);
- querySubmitted.set(true);
- List<String> params = new ArrayList<String>(3);
- params.add(appAttemptId.toString());
- params.add(query);
- params.add(plan.toJson());
- return QueryHeartbeatResponse.ResponseCommand.newBuilder()
- .setCommand("executeQuery")
- .addAllParams(params)
- .build();
- } else {
- return null;
- }
- }
-
- private boolean isFinishState(TajoProtos.QueryState state) {
- return state == TajoProtos.QueryState.QUERY_FAILED ||
- state == TajoProtos.QueryState.QUERY_KILLED ||
- state == TajoProtos.QueryState.QUERY_SUCCEEDED;
- }
-
- private void startCheckingQueryMasterStop() {
- Thread t = new Thread() {
- public void run() {
- try {
- ApplicationReport report = monitorApplication(appId,
- EnumSet.of(
- YarnApplicationState.FINISHED,
- YarnApplicationState.KILLED,
- YarnApplicationState.FAILED));
- queryMasterStopped.set(true);
- LOG.info("QueryMaster (" + queryId + ") stopped");
- } catch (YarnRemoteException e) {
- LOG.error(e.getMessage(), e);
- }
- }
- };
-
- t.start();
- }
-
- private ApplicationAttemptId allocateAndLaunchQueryMaster() throws YarnRemoteException {
- LOG.info("Allocate and launch QueryMaster:" + yarnClient);
- ApplicationSubmissionContext appContext = Records.newRecord(ApplicationSubmissionContext.class);
-
- // set the application id
- appContext.setApplicationId(appId);
- // set the application name
- appContext.setApplicationName("Tajo");
-
- Priority pri = Records.newRecord(Priority.class);
- pri.setPriority(5);
- appContext.setPriority(pri);
-
- // Set the queue to which this application is to be submitted in the RM
- appContext.setQueue("default");
-
- ContainerLaunchContext commonContainerLaunchContext =
- ContainerProxy.createCommonContainerLaunchContext(masterContext.getConf(), queryId.toString(), true);
-
- // Setup environment by cloning from common env.
- Map<String, String> env = commonContainerLaunchContext.getEnvironment();
- Map<String, String> myEnv = new HashMap<String, String>(env.size());
- myEnv.putAll(env);
-
- ////////////////////////////////////////////////////////////////////////////
- // Set the local resources
- ////////////////////////////////////////////////////////////////////////////
- // Set the necessary command to execute the application master
- Vector<CharSequence> vargs = new Vector<CharSequence>(30);
-
- // Set java executable command
- //LOG.info("Setting up app master command");
- vargs.add("${JAVA_HOME}" + "/bin/java");
- // Set Xmx based on am memory size
- vargs.add("-Xmx2000m");
- // Set Remote Debugging
- //if (!context.getQuery().getSubQuery(event.getSubQueryId()).isLeafQuery()) {
- //vargs.add("-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=5005");
- //}
- // Set class name
- vargs.add(QueryMasterRunner.class.getCanonicalName());
- vargs.add(queryId.toString()); // queryId
- vargs.add(String.valueOf(appSubmitTime));
- vargs.add(masterContext.getQueryMasterManagerService().getBindAddress().getHostName() + ":" +
- masterContext.getQueryMasterManagerService().getBindAddress().getPort());
-
- vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout");
- vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr");
-
- // Get final commmand
- StringBuilder command = new StringBuilder();
- for (CharSequence str : vargs) {
- command.append(str).append(" ");
- }
-
- LOG.info("Completed setting up QueryMasterRunner command " + command.toString());
- List<String> commands = new ArrayList<String>();
- commands.add(command.toString());
-
- final Resource resource = Records.newRecord(Resource.class);
- // TODO - get default value from conf
- resource.setMemory(2000);
- resource.setVirtualCores(1);
-
- Map<String, ByteBuffer> myServiceData = new HashMap<String, ByteBuffer>();
-
- ContainerLaunchContext masterContainerContext = BuilderUtils.newContainerLaunchContext(
- null, commonContainerLaunchContext.getUser(),
- resource, commonContainerLaunchContext.getLocalResources(), myEnv, commands,
- myServiceData, null, new HashMap<ApplicationAccessType, String>(2));
-
- appContext.setAMContainerSpec(masterContainerContext);
-
- LOG.info("Submitting QueryMaster to ResourceManager");
- yarnClient.submitApplication(appContext);
-
- ApplicationReport appReport = monitorApplication(appId, EnumSet.of(YarnApplicationState.ACCEPTED));
- ApplicationAttemptId attemptId = appReport.getCurrentApplicationAttemptId();
-
- LOG.info("Launching QueryMaster with id: " + attemptId);
-
- state = TajoProtos.QueryState.QUERY_MASTER_LAUNCHED;
-
- return attemptId;
- }
-
- private ApplicationReport monitorApplication(ApplicationId appId,
- Set<YarnApplicationState> finalState) throws YarnRemoteException {
-
- long sleepTime = 100;
- int count = 1;
- while (true) {
- // Get application report for the appId we are interested in
- ApplicationReport report = yarnClient.getApplicationReport(appId);
-
- LOG.info("Got application report from ASM for" + ", appId="
- + appId.getId() + ", appAttemptId="
- + report.getCurrentApplicationAttemptId() + ", clientToken="
- + report.getClientToken() + ", appDiagnostics="
- + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
- + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
- + report.getRpcPort() + ", appStartTime=" + report.getStartTime()
- + ", yarnAppState=" + report.getYarnApplicationState().toString()
- + ", distributedFinalState="
- + report.getFinalApplicationStatus().toString() + ", appTrackingUrl="
- + report.getTrackingUrl() + ", appUser=" + report.getUser());
-
- YarnApplicationState state = report.getYarnApplicationState();
- if (finalState.contains(state)) {
- return report;
- }
- try {
- Thread.sleep(sleepTime);
- sleepTime = count * 100;
- if(count < 10) {
- count++;
- }
- } catch (InterruptedException e) {
- //LOG.debug("Thread sleep in monitoring loop interrupted");
- }
-
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
deleted file mode 100644
index 65f237c..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.querymaster;
-
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol.QueryHeartbeat;
-import org.apache.tajo.ipc.QueryMasterManagerProtocol.QueryHeartbeatResponse;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.rpc.ProtoBlockingRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
-import org.apache.tajo.util.NetUtils;
-
-import java.net.InetSocketAddress;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-
-public class QueryMasterManagerService extends AbstractService {
- private final static Log LOG = LogFactory.getLog(QueryMasterManagerService.class);
-
- private final TajoMaster.MasterContext context;
- private final TajoConf conf;
- private final QueryMasterManagerProtocolServiceHandler masterHandler;
- private ProtoBlockingRpcServer server;
- private InetSocketAddress bindAddress;
-
- private final BoolProto BOOL_TRUE = BoolProto.newBuilder().setValue(true).build();
- private final BoolProto BOOL_FALSE = BoolProto.newBuilder().setValue(false).build();
-
- public QueryMasterManagerService(TajoMaster.MasterContext context) {
- super(QueryMasterManagerService.class.getName());
- this.context = context;
- this.conf = context.getConf();
- this.masterHandler = new QueryMasterManagerProtocolServiceHandler();
- }
-
- @Override
- public void start() {
- // TODO resolve hostname
- String confMasterServiceAddr = conf.getVar(ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS);
- InetSocketAddress initIsa = NetUtils.createSocketAddr(confMasterServiceAddr);
- try {
- server = new ProtoBlockingRpcServer(QueryMasterManagerProtocol.class, masterHandler, initIsa);
- } catch (Exception e) {
- LOG.error(e);
- }
- server.start();
- bindAddress = NetUtils.getConnectAddress(server.getListenAddress());
- this.conf.setVar(ConfVars.QUERY_MASTER_MANAGER_SERVICE_ADDRESS, NetUtils.normalizeInetSocketAddress(bindAddress));
- LOG.info("QueryMasterManagerService startup");
- super.start();
- }
-
- @Override
- public void stop() {
- if(server != null) {
- server.shutdown();
- server = null;
- }
- LOG.info("QueryMasterManagerService shutdown");
- super.stop();
- }
-
- public InetSocketAddress getBindAddress() {
- return bindAddress;
- }
-
- public class QueryMasterManagerProtocolServiceHandler implements QueryMasterManagerProtocol.QueryMasterManagerProtocolService.BlockingInterface {
- @Override
- public QueryHeartbeatResponse queryHeartbeat(RpcController controller, QueryHeartbeat request) throws ServiceException {
- // TODO - separate QueryMasterManagerProtocol, ClientServiceProtocol
- QueryId queryId = new QueryId(request.getQueryId());
- if(LOG.isDebugEnabled()) {
- LOG.debug("Received QueryHeartbeat:" + queryId + "," + request);
- }
- QueryMasterManager queryMasterManager = context.getQuery(queryId);
- if (queryMasterManager == null) {
- LOG.warn("No query:" + queryId);
- return QueryHeartbeatResponse.newBuilder().setHeartbeatResult(BOOL_FALSE).build();
- }
-
- QueryHeartbeatResponse.ResponseCommand command = queryMasterManager.queryHeartbeat(request);
-
- //ApplicationAttemptId attemptId = queryMasterManager.getAppAttemptId();
- //String attemptIdStr = attemptId == null ? null : attemptId.toString();
- QueryHeartbeatResponse.Builder builder = QueryHeartbeatResponse.newBuilder();
- builder.setHeartbeatResult(BOOL_TRUE);
- if(command != null) {
- builder.setResponseCommand(command);
- }
- return builder.build();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
index daab9fd..5a79464 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
@@ -39,13 +39,11 @@ public class QueryMasterRunner extends AbstractService {
private QueryConf queryConf;
private QueryMaster queryMaster;
private QueryId queryId;
- private long appSubmitTime;
private String queryMasterManagerAddress;
- public QueryMasterRunner(QueryId queryId, long appSubmitTime, String queryMasterManagerAddress) {
+ public QueryMasterRunner(QueryId queryId, String queryMasterManagerAddress) {
super(QueryMasterRunner.class.getName());
this.queryId = queryId;
- this.appSubmitTime = appSubmitTime;
this.queryMasterManagerAddress = queryMasterManagerAddress;
}
@@ -72,7 +70,7 @@ public class QueryMasterRunner extends AbstractService {
@Override
public void start() {
//create QueryMaster
- QueryMaster query = new QueryMaster(queryId, appSubmitTime, queryMasterManagerAddress);
+ QueryMaster query = new QueryMaster(null);
query.init(queryConf);
query.start();
@@ -90,13 +88,12 @@ public class QueryMasterRunner extends AbstractService {
UserGroupInformation.setConfiguration(conf);
- final QueryId queryId = TajoIdUtils.createQueryId(args[0]);
- final long appSubmitTime = Long.parseLong(args[1]);
- final String queryMasterManagerAddr = args[2];
+ final QueryId queryId = TajoIdUtils.parseQueryId(args[0]);
+ final String queryMasterManagerAddr = args[1];
LOG.info("Received QueryId:" + queryId);
- QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, appSubmitTime, queryMasterManagerAddr);
+ QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, queryMasterManagerAddr);
queryMasterRunner.init(conf);
queryMasterRunner.start();
[8/8] git commit: TAJO-127: Implement Tajo Resource Manager.
(hyoungjunkim via hyunsik)
Posted by hy...@apache.org.
TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)
Project: http://git-wip-us.apache.org/repos/asf/incubator-tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tajo/commit/d48f2667
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/d48f2667
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/d48f2667
Branch: refs/heads/master
Commit: d48f2667b822564260b6e5c9705a41e5dcd1c4fc
Parents: 8b8b668
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Aug 26 21:18:23 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Aug 26 21:21:15 2013 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 +
pom.xml | 9 +
.../org/apache/tajo/catalog/CatalogServer.java | 4 +-
.../java/org/apache/tajo/ExecutionBlockId.java | 97 +++
.../src/main/java/org/apache/tajo/QueryId.java | 180 +----
.../java/org/apache/tajo/QueryIdFactory.java | 96 ++-
.../org/apache/tajo/QueryUnitAttemptId.java | 144 +---
.../main/java/org/apache/tajo/QueryUnitId.java | 162 +---
.../main/java/org/apache/tajo/SubQueryId.java | 166 ----
.../java/org/apache/tajo/conf/TajoConf.java | 3 +-
.../main/java/org/apache/tajo/util/TUtil.java | 6 +-
.../java/org/apache/tajo/util/TajoIdUtils.java | 100 +--
tajo-common/src/main/proto/TajoIdProtos.proto | 11 +-
.../org/apache/tajo/datum/TestBitDatum.java | 2 +-
tajo-core/tajo-core-backend/pom.xml | 5 +-
.../src/main/java/log4j.properties | 3 +
.../main/java/org/apache/tajo/cli/TajoCli.java | 15 +-
.../java/org/apache/tajo/client/TajoClient.java | 24 +-
.../tajo/engine/query/QueryUnitRequestImpl.java | 4 +-
.../apache/tajo/engine/query/ResultSetImpl.java | 41 +-
.../ipc/protocolrecords/QueryUnitRequest.java | 6 +-
.../org/apache/tajo/master/ContainerProxy.java | 409 +---------
.../org/apache/tajo/master/ExecutionBlock.java | 12 +-
.../org/apache/tajo/master/GlobalEngine.java | 244 ++----
.../org/apache/tajo/master/GlobalPlanner.java | 38 +-
.../apache/tajo/master/TajoAsyncDispatcher.java | 234 ++++++
.../apache/tajo/master/TajoContainerProxy.java | 163 ++++
.../java/org/apache/tajo/master/TajoMaster.java | 118 +--
.../tajo/master/TajoMasterClientService.java | 101 ++-
.../apache/tajo/master/TajoMasterService.java | 170 ++++
.../tajo/master/TaskRunnerGroupEvent.java | 12 +-
.../tajo/master/TaskRunnerLauncherImpl.java | 171 ----
.../apache/tajo/master/TaskSchedulerImpl.java | 71 +-
.../apache/tajo/master/YarnContainerProxy.java | 446 ++++++++++
.../tajo/master/YarnTaskRunnerLauncherImpl.java | 208 +++++
.../master/event/ContainerAllocationEvent.java | 20 +-
.../event/GrouppedContainerAllocatorEvent.java | 6 +-
.../tajo/master/event/QueryStartEvent.java | 50 ++
.../tajo/master/event/QuerySubQueryEvent.java | 12 +-
.../master/event/SubQueryCompletedEvent.java | 14 +-
.../event/SubQueryContainerAllocationEvent.java | 4 +-
.../apache/tajo/master/event/SubQueryEvent.java | 8 +-
.../tajo/master/event/SubQuerySucceeEvent.java | 4 +-
.../tajo/master/event/SubQueryTaskEvent.java | 2 +-
.../event/TaskAttemptStatusUpdateEvent.java | 2 +-
.../tajo/master/event/TaskCompletionEvent.java | 2 +-
.../tajo/master/event/TaskFatalErrorEvent.java | 2 +-
.../tajo/master/event/TaskRequestEvent.java | 11 +-
.../tajo/master/event/TaskScheduleEvent.java | 2 +-
.../tajo/master/event/TaskSchedulerEvent.java | 12 +-
.../apache/tajo/master/querymaster/Query.java | 54 +-
.../master/querymaster/QueryInProgress.java | 285 +++++++
.../tajo/master/querymaster/QueryInfo.java | 127 +++
.../tajo/master/querymaster/QueryJobEvent.java | 43 +
.../master/querymaster/QueryJobManager.java | 172 ++++
.../tajo/master/querymaster/QueryMaster.java | 809 ++++---------------
.../querymaster/QueryMasterClientService.java | 197 -----
.../master/querymaster/QueryMasterManager.java | 353 --------
.../querymaster/QueryMasterManagerService.java | 116 ---
.../master/querymaster/QueryMasterRunner.java | 13 +-
.../master/querymaster/QueryMasterTask.java | 445 ++++++++++
.../tajo/master/querymaster/QueryUnit.java | 12 +-
.../master/querymaster/QueryUnitAttempt.java | 13 +-
.../tajo/master/querymaster/Repartitioner.java | 22 +-
.../tajo/master/querymaster/SubQuery.java | 48 +-
.../tajo/master/rm/RMContainerAllocator.java | 208 -----
.../tajo/master/rm/TajoWorkerContainer.java | 120 +++
.../tajo/master/rm/TajoWorkerContainerId.java | 47 ++
.../master/rm/TajoWorkerResourceManager.java | 394 +++++++++
.../apache/tajo/master/rm/WorkerResource.java | 194 +++++
.../tajo/master/rm/WorkerResourceManager.java | 60 ++
.../org/apache/tajo/master/rm/WorkerStatus.java | 25 +
.../master/rm/YarnRMContainerAllocator.java | 236 ++++++
.../tajo/master/rm/YarnTajoResourceManager.java | 337 ++++++++
.../apache/tajo/util/ApplicationIdUtils.java | 41 +
.../apache/tajo/webapp/StaticHttpServer.java | 26 +-
.../tajo/worker/AbstractResourceAllocator.java | 79 ++
.../apache/tajo/worker/ResourceAllocator.java | 27 +
.../tajo/worker/TajoResourceAllocator.java | 346 ++++++++
.../java/org/apache/tajo/worker/TajoWorker.java | 424 ++++++++++
.../tajo/worker/TajoWorkerClientService.java | 210 +++++
.../tajo/worker/TajoWorkerManagerService.java | 234 ++++++
.../main/java/org/apache/tajo/worker/Task.java | 43 +-
.../java/org/apache/tajo/worker/TaskRunner.java | 256 +++---
.../apache/tajo/worker/TaskRunnerManager.java | 108 +++
.../tajo/worker/YarnResourceAllocator.java | 106 +++
.../retriever/AdvancedDataRetriever.java | 10 +-
.../src/main/proto/ClientProtocol.proto | 12 +-
.../src/main/proto/ClientProtos.proto | 10 +-
.../main/proto/QueryMasterClientProtocol.proto | 2 +-
.../main/proto/QueryMasterManagerProtocol.proto | 50 --
.../src/main/proto/QueryMasterProtocol.proto | 132 ---
.../src/main/proto/TajoIdProtos.proto | 11 +-
.../main/proto/TajoMasterClientProtocol.proto | 5 +-
.../src/main/proto/TajoMasterProtocol.proto | 98 +++
.../src/main/proto/TajoWorkerProtocol.proto | 137 ++++
.../src/main/resources/log4j.properties | 3 +
.../src/main/resources/tajo-default.xml | 68 ++
.../src/test/java/log4j.properties | 3 +
.../apache/tajo/LocalTajoTestingUtility.java | 13 +-
.../org/apache/tajo/MiniTajoYarnCluster.java | 3 -
.../org/apache/tajo/TajoTestingCluster.java | 120 +--
.../org/apache/tajo/TestQueryIdFactory.java | 7 +-
.../test/java/org/apache/tajo/TestTajoIds.java | 78 +-
.../test/java/org/apache/tajo/TpchTestBase.java | 17 +-
.../plan/global/TestGlobalQueryPlanner.java | 1 -
.../global/TestGlobalQueryOptimizer.java | 15 +-
.../planner/physical/TestPhysicalPlanner.java | 2 -
.../apache/tajo/master/TestRepartitioner.java | 7 +-
.../org/apache/tajo/worker/TaskRunnerTest.java | 18 +-
.../tajo/worker/TestRangeRetrieverHandler.java | 2 -
.../worker/dataserver/TestHttpDataServer.java | 12 +-
.../src/test/resources/tajo-default.xml | 6 +
tajo-core/tajo-core-pullserver/pom.xml | 11 +-
.../tajo/pullserver/HttpDataServerHandler.java | 9 +-
.../tajo/pullserver/PullServerAuxService.java | 15 +-
.../tajo/pullserver/TajoPullServerService.java | 652 +++++++++++++++
.../retriever/AdvancedDataRetriever.java | 23 +-
.../java/org/apache/tajo/storage/CSVFile.java | 187 ++---
.../org/apache/tajo/storage/MergeScanner.java | 8 +-
tajo-dist/src/main/bin/start-tajo.sh | 8 +
tajo-dist/src/main/bin/stop-tajo.sh | 11 +-
tajo-dist/src/main/bin/tajo | 4 +
tajo-dist/src/main/bin/tajo-config.sh | 15 +-
tajo-dist/src/main/bin/tajo-daemon.sh | 12 +-
tajo-dist/src/main/bin/tajo-daemons.sh | 68 ++
tajo-dist/src/main/conf/tajo-env.sh | 13 +-
tajo-dist/src/main/conf/workers | 1 +
.../org/apache/tajo/rpc/NettyServerBase.java | 1 -
.../apache/tajo/rpc/ProtoAsyncRpcClient.java | 5 +-
.../apache/tajo/rpc/ProtoAsyncRpcServer.java | 3 +-
.../apache/tajo/rpc/ProtoBlockingRpcClient.java | 12 +-
.../java/org/apache/tajo/util/NetUtils.java | 2 +-
133 files changed, 7812 insertions(+), 3948 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index de7d746..c6b3f84 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,8 @@ Release 0.2.0 - unreleased
IMPROVEMENTS
+ TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via hyunsik)
+
TAJO-84: Task scheduling with considering disk load balance. (jinho)
TAJO-123: Clean up the logical plan's json format. (hyunsik)
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e1e7b7d..07b05a1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -91,6 +91,14 @@
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-report-plugin</artifactId>
+ <version>2.15</version>
+ <configuration>
+ <aggregate>true</aggregate>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<version>2.5.1</version>
<configuration>
@@ -297,6 +305,7 @@
<numUnapprovedLicenses>0</numUnapprovedLicenses>
<excludes>
<exclude>CHANGES.txt</exclude>
+ <exclude>**/workers</exclude>
<exclude>**/*.tql</exclude>
<exclude>**/*.sql</exclude>
<exclude>**/*.schema</exclude>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
index dd36c3e..f0a0584 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
@@ -128,7 +128,9 @@ public class CatalogServer extends AbstractService {
// Creation of a HSA will force a resolve.
InetSocketAddress initIsa = NetUtils.createSocketAddr(serverAddr);
try {
- this.rpcServer = new ProtoBlockingRpcServer(CatalogProtocol.class, handler, initIsa);
+ this.rpcServer = new ProtoBlockingRpcServer(
+ CatalogProtocol.class,
+ handler, initIsa);
this.rpcServer.start();
this.bindAddress = NetUtils.getConnectAddress(this.rpcServer.getListenAddress());
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java b/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
new file mode 100644
index 0000000..2dc4441
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+public class ExecutionBlockId implements Comparable<ExecutionBlockId> {
+ public static final String EB_ID_PREFIX = "eb";
+ private QueryId queryId;
+ private int id;
+
+ public ExecutionBlockId(QueryId queryId, int id) {
+ this.queryId = queryId;
+ this.id = id;
+ }
+
+ public ExecutionBlockId(TajoIdProtos.ExecutionBlockIdProto proto) {
+ this(new QueryId(proto.getQueryId()), proto.getId());
+ }
+
+// public ExecutionBlockId(String idStr) {
+// String[] tokens = idStr.split(QueryId.SEPARATOR);
+// if(tokens.length < 3) {
+// throw new RuntimeException("Wrong ExecutionBlockId format[" + idStr + "]");
+// }
+//
+// this.queryId = new QueryId(tokens);
+// this.id = Integer.parseInt(tokens[3]);
+// }
+
+ @Override
+ public String toString() {
+ return EB_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
+ }
+
+ @Override
+ public int compareTo(ExecutionBlockId executionBlockId) {
+ int result = queryId.compareTo(executionBlockId.queryId);
+ if (result == 0) {
+ return id - executionBlockId.id;
+ } else {
+ return result;
+ }
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (this == obj) {
+ return true;
+ }
+ if(!(obj instanceof ExecutionBlockId)) {
+ return false;
+ }
+ return compareTo((ExecutionBlockId)obj) == 0;
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ public TajoIdProtos.ExecutionBlockIdProto getProto() {
+ return TajoIdProtos.ExecutionBlockIdProto.newBuilder()
+ .setQueryId(queryId.getProto())
+ .setId(id)
+ .build();
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ public String toStringNoPrefix() {
+ return queryId.toStringNoPrefix() + QueryId.SEPARATOR + QueryIdFactory.EB_ID_FORMAT.format(id);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryId.java b/tajo-common/src/main/java/org/apache/tajo/QueryId.java
index 5dbbaca..af1ae56 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryId.java
@@ -18,172 +18,72 @@
package org.apache.tajo;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
-
-import java.text.NumberFormat;
-
-/**
- * QueryId represents a unique identifier of a query.
- */
public class QueryId implements Comparable<QueryId> {
- public static final String PREFIX = "q";
- public static final String SEPARATOR = "_";
-
- ApplicationAttemptIdProto proto = ApplicationAttemptIdProto
- .getDefaultInstance();
- ApplicationAttemptIdProto.Builder builder = null;
- boolean viaProto = false;
+ public static String SEPARATOR = "_";
+ public static final String QUERY_ID_PREFIX = "q";
- private ApplicationId applicationId = null;
+ private String id;
+ private int seq;
- public QueryId() {
- builder = ApplicationAttemptIdProto.newBuilder();
+ public QueryId(String id, int seq) {
+ this.id = id;
+ this.seq = seq;
}
- public QueryId(ApplicationAttemptIdProto proto) {
- this.proto = proto;
- viaProto = true;
+ public QueryId(TajoIdProtos.QueryIdProto queryId) {
+ this(queryId.getId(), queryId.getSeq());
}
- public synchronized ApplicationAttemptIdProto getProto() {
- mergeLocalToProto();
- proto = viaProto ? proto : builder.build();
- viaProto = true;
- return proto;
+ public String getId() {
+ return id;
}
- private synchronized void mergeLocalToBuilder() {
- if (this.applicationId != null
- && !((ApplicationIdPBImpl) applicationId).getProto().equals(
- builder.getApplicationId())) {
- builder.setApplicationId(convertToProtoFormat(this.applicationId));
- }
+ public int getSeq() {
+ return seq;
}
- private synchronized void mergeLocalToProto() {
- if (viaProto)
- maybeInitBuilder();
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
+ @Override
+ public String toString() {
+ return QUERY_ID_PREFIX + SEPARATOR + toStringNoPrefix();
}
- private synchronized void maybeInitBuilder() {
- if (viaProto || builder == null) {
- builder = ApplicationAttemptIdProto.newBuilder(proto);
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
}
- viaProto = false;
- }
-
- public synchronized int getAttemptId() {
- ApplicationAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
- return (p.getAttemptId());
- }
-
- public synchronized void setAttemptId(int attemptId) {
- maybeInitBuilder();
- builder.setAttemptId((attemptId));
- }
-
- public synchronized ApplicationId getApplicationId() {
- ApplicationAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
- if (this.applicationId != null) {
- return this.applicationId;
+ if (this == obj) {
+ return true;
}
- if (!p.hasApplicationId()) {
- return null;
+ if(!(obj instanceof QueryId)) {
+ return false;
}
- this.applicationId = convertFromProtoFormat(p.getApplicationId());
- return this.applicationId;
- }
-
- public synchronized void setApplicationId(ApplicationId appId) {
- maybeInitBuilder();
- if (appId == null)
- builder.clearApplicationId();
- this.applicationId = appId;
- }
-
- private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
- return new ApplicationIdPBImpl(p);
+ return compareTo((QueryId)obj) == 0;
}
- public static ApplicationIdProto convertToProtoFormat(ApplicationId t) {
- return ((ApplicationIdPBImpl)t).getProto();
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
}
@Override
public int compareTo(QueryId queryId) {
- int compVal = getApplicationId().compareTo(queryId.getApplicationId());
- if (compVal != 0) {
- return compVal;
+ int result = id.compareTo(queryId.id);
+ if (result == 0) {
+ return seq - queryId.seq;
} else {
- return getAttemptId() - queryId.getAttemptId();
+ return result;
}
}
- static final ThreadLocal<NumberFormat> appIdFormat =
- new ThreadLocal<NumberFormat>() {
- @Override
- public NumberFormat initialValue() {
- NumberFormat fmt = NumberFormat.getInstance();
- fmt.setGroupingUsed(false);
- fmt.setMinimumIntegerDigits(4);
- return fmt;
- }
- };
-
- static final ThreadLocal<NumberFormat> attemptIdFormat =
- new ThreadLocal<NumberFormat>() {
- @Override
- public NumberFormat initialValue() {
- NumberFormat fmt = NumberFormat.getInstance();
- fmt.setGroupingUsed(false);
- fmt.setMinimumIntegerDigits(6);
- return fmt;
- }
- };
-
- @Override
- public int hashCode() {
- // Generated by eclipse.
- final int prime = 31;
- int result = 1;
- ApplicationId appId = getApplicationId();
- result = prime * result + appId.hashCode();
- result = prime * result + getAttemptId();
- return result;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (this == obj)
- return true;
- if (obj == null)
- return false;
- if (getClass() != obj.getClass())
- return false;
- QueryId other = (QueryId) obj;
- if (!this.getApplicationId().equals(other.getApplicationId()))
- return false;
- if (this.getAttemptId() != other.getAttemptId())
- return false;
- return true;
+ public TajoIdProtos.QueryIdProto getProto() {
+ return TajoIdProtos.QueryIdProto.newBuilder()
+ .setId(id)
+ .setSeq(seq)
+ .build();
}
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder(PREFIX).append(SEPARATOR)
- .append(this.getApplicationId().getClusterTimestamp()).append(SEPARATOR)
- .append(appIdFormat.get().format(this.getApplicationId().getId()))
- .append(SEPARATOR)
- .append(attemptIdFormat.get().format(getAttemptId()));
- return sb.toString();
+ public String toStringNoPrefix() {
+ return id + SEPARATOR + QueryIdFactory.ID_FORMAT.format(seq);
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java b/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
index b1a6ab4..90533e3 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
@@ -18,39 +18,97 @@
package org.apache.tajo;
-import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.tajo.util.TajoIdUtils;
+import java.text.DecimalFormat;
+import java.util.HashMap;
+import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
public class QueryIdFactory {
- private static AtomicInteger nextId =
- new AtomicInteger(-1);
-
- public static void reset() {
- nextId.set(-1);
+ public static final QueryId NULL_QUERY_ID = newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0), 0);
+
+ public static DecimalFormat ID_FORMAT = new DecimalFormat("0000");
+
+ public static DecimalFormat EB_ID_FORMAT = new DecimalFormat("000000");
+
+ public static DecimalFormat QU_ID_FORMAT = new DecimalFormat("000000");
+
+ public static DecimalFormat ATTEMPT_ID_FORMAT = new DecimalFormat("00");
+
+ private static Map<String, AtomicInteger> queryNexIdMap = new HashMap<String, AtomicInteger>();
+
+ private static AtomicInteger nextId = new AtomicInteger(0);
+
+ /*
+ * <ul>
+ * <li>QueryId == q_{masterId}_{seq}</li>
+ * <li>masterId == tajoMasterId or YarnResourceManagerId</li>
+ * <li>seq = TajoSeq or YarnSeq</li>
+ * </ul>
+ */
+ public synchronized static QueryId newQueryId(String seedQueryId) {
+ AtomicInteger queryNextId = queryNexIdMap.get(seedQueryId);
+ if(queryNextId == null) {
+ queryNextId = new AtomicInteger(0);
+ queryNexIdMap.put(seedQueryId, queryNextId);
+ }
+ if(isYarnId(seedQueryId)) {
+ String[] tokens = seedQueryId.split("_");
+ return new QueryId(tokens[1], Integer.parseInt(tokens[2]));
+ } else {
+ int seq = queryNextId.incrementAndGet();
+ if(seq >= 10000) {
+ queryNextId.set(0);
+ seq = queryNextId.incrementAndGet();
+ }
+
+ return new QueryId(seedQueryId, seq);
+ }
+ }
+
+ public synchronized static QueryId newQueryId(long timestamp, int seq) {
+ return new QueryId(String.valueOf(timestamp), seq);
}
+ /**
+ * for test
+ * @return
+ */
public synchronized static QueryId newQueryId() {
- int idInt = nextId.incrementAndGet();
- return TajoIdUtils.createQueryId(BuilderUtils.newApplicationId(
- System.currentTimeMillis(), idInt), idInt);
+ return newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0));
}
-
- public synchronized static SubQueryId newSubQueryId(QueryId queryId) {
- return TajoIdUtils.createSubQueryId(queryId, nextId.incrementAndGet());
+
+ public synchronized static QueryId newQueryId(String seedQueryId, int seq) {
+ if(isYarnId(seedQueryId)) {
+ String[] tokens = seedQueryId.split("_");
+ return new QueryId(tokens[1], Integer.parseInt(tokens[2]));
+ } else {
+ return new QueryId(seedQueryId, seq);
+ }
}
-
- public synchronized static QueryUnitId newQueryUnitId(SubQueryId subQueryId) {
- return new QueryUnitId(subQueryId, nextId.incrementAndGet());
+
+ private static boolean isYarnId(String id) {
+ return id.startsWith("application");
+ }
+
+ public synchronized static ExecutionBlockId newExecutionBlockId(QueryId queryId) {
+ return new ExecutionBlockId(queryId, nextId.incrementAndGet());
+ }
+
+ public synchronized static ExecutionBlockId newExecutionBlockId(QueryId queryId, int id) {
+ return new ExecutionBlockId(queryId, id);
+ }
+
+ public synchronized static QueryUnitId newQueryUnitId(ExecutionBlockId executionBlockId) {
+ return new QueryUnitId(executionBlockId, nextId.incrementAndGet());
}
- public synchronized static QueryUnitId newQueryUnitId(SubQueryId subQueryId, int taskId) {
- return new QueryUnitId(subQueryId, taskId);
+ public synchronized static QueryUnitId newQueryUnitId(ExecutionBlockId executionBlockId, int id) {
+ return new QueryUnitId(executionBlockId, id);
}
- public synchronized static QueryUnitAttemptId newQueryUnitAttemptId(
- final QueryUnitId queryUnitId, final int attemptId) {
+ public synchronized static QueryUnitAttemptId newQueryUnitAttemptId(QueryUnitId queryUnitId, final int attemptId) {
return new QueryUnitAttemptId(queryUnitId, attemptId);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java b/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
index 3ef436e..98ba5d1 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
@@ -18,139 +18,75 @@
package org.apache.tajo;
-import com.google.common.base.Objects;
-import org.apache.tajo.TajoIdProtos.QueryUnitAttemptIdProto;
-import org.apache.tajo.TajoIdProtos.QueryUnitAttemptIdProtoOrBuilder;
-import org.apache.tajo.common.ProtoObject;
+public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId> {
+ public static final String QUA_ID_PREFIX = "ta";
-import java.text.NumberFormat;
+ private QueryUnitId queryUnitId;
+ private int id;
-public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId>, ProtoObject<QueryUnitAttemptIdProto> {
- private static final String PREFIX="ta";
-
- private static final NumberFormat format = NumberFormat.getInstance();
- static {
- format.setGroupingUsed(false);
- format.setMinimumIntegerDigits(2);
+ public QueryUnitId getQueryUnitId() {
+ return queryUnitId;
}
- private QueryUnitId queryUnitId = null;
- private int id = -1;
- private String finalId = null;
-
- private QueryUnitAttemptIdProto proto =
- QueryUnitAttemptIdProto.getDefaultInstance();
- private QueryUnitAttemptIdProto.Builder builder = null;
- private boolean viaProto = false;
-
- public QueryUnitAttemptId() {
- builder = QueryUnitAttemptIdProto.newBuilder();
+ public int getId() {
+ return id;
}
- public QueryUnitAttemptId(final QueryUnitId queryUnitId, final int id) {
- this.queryUnitId = queryUnitId;
+ public void setId(int id) {
this.id = id;
}
- public QueryUnitAttemptId(QueryUnitAttemptIdProto proto) {
- this.proto = proto;
- viaProto = true;
- }
-
- public QueryUnitAttemptId(final String finalId) {
- this.finalId = finalId;
- int i = finalId.lastIndexOf(QueryId.SEPARATOR);
- this.queryUnitId = new QueryUnitId(finalId.substring(0, i));
- this.id = Integer.valueOf(finalId.substring(i+1));
- }
-
- public int getId() {
- QueryUnitAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
- if (this.id != -1) {
- return this.id;
- }
- if (!p.hasId()) {
- return -1;
- }
- this.id = p.getId();
- return id;
- }
-
- public QueryUnitId getQueryUnitId() {
- QueryUnitAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
- if (this.queryUnitId != null) {
- return this.queryUnitId;
- }
- if (!p.hasId()) {
- return null;
- }
- this.queryUnitId = new QueryUnitId(p.getQueryUnitId());
- return queryUnitId;
+ public QueryUnitAttemptId(QueryUnitId queryUnitId, int id) {
+ this.queryUnitId = queryUnitId;
+ this.id = id;
}
- public QueryId getQueryId() {
- return this.getQueryUnitId().getQueryId();
+ public QueryUnitAttemptId(TajoIdProtos.QueryUnitAttemptIdProto proto) {
+ this(new QueryUnitId(proto.getQueryUnitId()), proto.getId());
}
- public SubQueryId getSubQueryId() {
- return this.getQueryUnitId().getSubQueryId();
+ public TajoIdProtos.QueryUnitAttemptIdProto getProto() {
+ return TajoIdProtos.QueryUnitAttemptIdProto.newBuilder()
+ .setQueryUnitId(queryUnitId.getProto())
+ .setId(id)
+ .build();
}
@Override
- public final String toString() {
- if (finalId == null) {
- StringBuilder sb = new StringBuilder(PREFIX);
- SubQueryId subQueryId = getQueryUnitId().getSubQueryId();
- QueryId appId = subQueryId.getQueryId();
- sb.append(QueryId.SEPARATOR).append(appId.getApplicationId().getClusterTimestamp())
- .append(QueryId.SEPARATOR).append(QueryId.appIdFormat.get().format(appId.getApplicationId().getId()))
- .append(QueryId.SEPARATOR).append(QueryId.attemptIdFormat.get().format(appId.getAttemptId()))
- .append(QueryId.SEPARATOR).append(SubQueryId.subQueryIdFormat.get().format(subQueryId.getId()))
- .append(QueryId.SEPARATOR).append(QueryUnitId.queryUnitIdFormat.get().format(getQueryUnitId().getId()))
- .append(QueryId.SEPARATOR).append(format.format(getId()));
- finalId = sb.toString();
+ public int compareTo(QueryUnitAttemptId queryUnitAttemptId) {
+ int result = queryUnitId.compareTo(queryUnitAttemptId.queryUnitId);
+ if (result == 0) {
+ return id - queryUnitAttemptId.id;
+ } else {
+ return result;
}
- return this.finalId;
}
@Override
- public final boolean equals(final Object o) {
- if (o instanceof QueryUnitAttemptId) {
- QueryUnitAttemptId other = (QueryUnitAttemptId) o;
- return this.toString().equals(other.toString());
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (this == obj) {
+ return true;
}
- return false;
+ if(!(obj instanceof QueryUnitAttemptId)) {
+ return false;
+ }
+ return compareTo((QueryUnitAttemptId)obj) == 0;
}
@Override
public int hashCode() {
- return Objects.hashCode(getQueryUnitId(), getId());
+ return toString().hashCode();
}
@Override
- public int compareTo(QueryUnitAttemptId o) {
- return this.getId() - o.getId();
- }
-
- private void mergeLocalToBuilder() {
- if (builder == null) {
- builder = QueryUnitAttemptIdProto.newBuilder(proto);
- }
- if (this.queryUnitId != null) {
- builder.setQueryUnitId(queryUnitId.getProto());
- }
- if (this.id != -1) {
- builder.setId(id);
- }
+ public String toString() {
+ return QUA_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
}
- @Override
- public QueryUnitAttemptIdProto getProto() {
- if (!viaProto) {
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
- }
- return proto;
+ public String toStringNoPrefix() {
+ return queryUnitId.toStringNoPrefix() + QueryId.SEPARATOR + QueryIdFactory.ATTEMPT_ID_FORMAT.format(id);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java b/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
index 4826691..21addf9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
@@ -18,145 +18,71 @@
package org.apache.tajo;
-import com.google.common.base.Objects;
-import org.apache.tajo.TajoIdProtos.QueryUnitIdProto;
-import org.apache.tajo.TajoIdProtos.QueryUnitIdProtoOrBuilder;
-import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.util.TajoIdUtils;
+public class QueryUnitId implements Comparable<QueryUnitId> {
+ public static final String QU_ID_PREFIX = "t";
-import java.text.NumberFormat;
+ private ExecutionBlockId executionBlockId;
+ private int id;
-public class QueryUnitId implements Comparable<QueryUnitId>,
- ProtoObject<QueryUnitIdProto> {
- private static final String PREFIX = "t";
-
- static final ThreadLocal<NumberFormat> queryUnitIdFormat =
- new ThreadLocal<NumberFormat>() {
- @Override
- public NumberFormat initialValue() {
- NumberFormat fmt = NumberFormat.getInstance();
- fmt.setGroupingUsed(false);
- fmt.setMinimumIntegerDigits(6);
- return fmt;
- }
- };
-
- private SubQueryId subQueryId = null;
- private int id = -1;
- private String finalId = null;
-
- private QueryUnitIdProto proto = QueryUnitIdProto.getDefaultInstance();
- private QueryUnitIdProto.Builder builder = null;
- private boolean viaProto = false;
-
- public QueryUnitId() {
- builder = QueryUnitIdProto.newBuilder();
- }
-
- public QueryUnitId(final SubQueryId subQueryId,
- final int id) {
- this.subQueryId = subQueryId;
+ public QueryUnitId(ExecutionBlockId executionBlockId, int id) {
+ this.executionBlockId = executionBlockId;
this.id = id;
}
-
- public QueryUnitId(QueryUnitIdProto proto) {
- this.proto = proto;
- viaProto = true;
+
+ public QueryUnitId(TajoIdProtos.QueryUnitIdProto proto) {
+ this(new ExecutionBlockId(proto.getExecutionBlockId()), proto.getId());
}
-
- public QueryUnitId(final String finalId) {
- this.finalId = finalId;
- int i = finalId.lastIndexOf(QueryId.SEPARATOR);
- this.subQueryId = TajoIdUtils.newSubQueryId(finalId.substring(0, i));
- this.id = Integer.valueOf(finalId.substring(i+1));
+
+ public ExecutionBlockId getExecutionBlockId() {
+ return executionBlockId;
}
-
+
public int getId() {
- QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
- if (this.id != -1) {
- return this.id;
- }
- if (!p.hasId()) {
- return -1;
- }
- this.id = p.getId();
return id;
}
-
- public SubQueryId getSubQueryId() {
- QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
- if (this.subQueryId != null) {
- return this.subQueryId;
- }
- if (!p.hasSubQueryId()) {
- return null;
- }
- this.subQueryId = TajoIdUtils.newSubQueryId(p.getSubQueryId());
- return this.subQueryId;
- }
-
- public QueryId getQueryId() {
- return this.getSubQueryId().getQueryId();
+
+ public TajoIdProtos.QueryUnitIdProto getProto() {
+ return TajoIdProtos.QueryUnitIdProto.newBuilder()
+ .setExecutionBlockId(executionBlockId.getProto())
+ .setId(id)
+ .build();
}
-
+
@Override
- public final String toString() {
- if (finalId == null) {
- StringBuilder sb = new StringBuilder(PREFIX);
- QueryId appId = getSubQueryId().getQueryId();
- sb.append(QueryId.SEPARATOR).append(
- appId.getApplicationId().getClusterTimestamp())
- .append(QueryId.SEPARATOR).append(
- QueryId.appIdFormat.get().format(appId.getApplicationId().getId()))
- .append(QueryId.SEPARATOR).append(
- QueryId.attemptIdFormat.get().format(appId.getAttemptId()))
- .append(QueryId.SEPARATOR).append(
- SubQueryId.subQueryIdFormat.get().format(getSubQueryId().getId()))
- .append(QueryId.SEPARATOR).append(queryUnitIdFormat.get().format(getId()));
- finalId = sb.toString();
+ public int compareTo(QueryUnitId queryUnitId) {
+ int result = executionBlockId.compareTo(queryUnitId.executionBlockId);
+ if (result == 0) {
+ return id - queryUnitId.id;
+ } else {
+ return result;
}
- return this.finalId;
}
-
+
@Override
- public final boolean equals(final Object o) {
- if (o instanceof QueryUnitId) {
- QueryUnitId other = (QueryUnitId) o;
- return getSubQueryId().equals(other.getSubQueryId()) &&
- getId() == other.getId();
- }
- return false;
+ public boolean equals(Object obj) {
+ if (obj == null) {
+ return false;
+ }
+ if (this == obj) {
+ return true;
+ }
+ if(!(obj instanceof QueryUnitId)) {
+ return false;
+ }
+ return compareTo((QueryUnitId)obj) == 0;
}
-
+
@Override
public int hashCode() {
- return Objects.hashCode(getSubQueryId(), getId());
+ return toString().hashCode();
}
@Override
- public final int compareTo(final QueryUnitId o) {
- return this.toString().compareTo(o.toString());
- }
-
- private void mergeLocalToBuilder() {
- if (builder == null) {
- builder = QueryUnitIdProto.newBuilder(proto);
- }
- if (this.subQueryId != null) {
- builder.setSubQueryId(subQueryId.getProto());
- }
- if (this.id != -1) {
- builder.setId(id);
- }
+ public String toString() {
+ return QU_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
}
- @Override
- public QueryUnitIdProto getProto() {
- if (!viaProto) {
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
- }
- return proto;
+ public String toStringNoPrefix() {
+ return executionBlockId.toStringNoPrefix() + QueryId.SEPARATOR + QueryIdFactory.QU_ID_FORMAT.format(id);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java b/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java
deleted file mode 100644
index 2a11f38..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/SubQueryId.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo;
-
-import org.apache.tajo.TajoIdProtos.SubQueryIdProto;
-import org.apache.tajo.TajoIdProtos.SubQueryIdProtoOrBuilder;
-
-import java.text.NumberFormat;
-
-public class SubQueryId implements Comparable<SubQueryId> {
- public static final String PREFIX = "sq";
-
- static final ThreadLocal<NumberFormat> subQueryIdFormat =
- new ThreadLocal<NumberFormat>() {
- @Override
- public NumberFormat initialValue() {
- NumberFormat fmt = NumberFormat.getInstance();
- fmt.setGroupingUsed(false);
- fmt.setMinimumIntegerDigits(2);
- return fmt;
- }
- };
-
- private SubQueryIdProto proto = SubQueryIdProto.getDefaultInstance();
- private SubQueryIdProto.Builder builder = null;
- private boolean viaProto = false;
-
- private QueryId queryId = null;
-
- public SubQueryId() {
- builder = SubQueryIdProto.newBuilder(proto);
- }
-
- public SubQueryId(SubQueryIdProto proto) {
- this.proto = proto;
- viaProto = true;
- }
-
- /**
- * @return the subquery number.
- */
- public synchronized int getId() {
- SubQueryIdProtoOrBuilder p = viaProto ? proto : builder;
- return (p.getId());
- }
-
- public synchronized void setId(int id) {
- maybeInitBuilder();
- builder.setId((id));
- }
-
- /**
- * @return the associated <code>QueryId</code>
- */
- public synchronized QueryId getQueryId() {
- SubQueryIdProtoOrBuilder p = viaProto ? proto : builder;
- if (this.queryId != null) {
- return this.queryId;
- }
- if (!p.hasQueryId()) {
- return null;
- }
- queryId = new QueryId(p.getQueryId());
- return queryId;
- }
-
- public synchronized void setQueryId(QueryId queryId) {
- maybeInitBuilder();
- if (queryId == null)
- builder.clearQueryId();
- this.queryId = queryId;
- }
-
- @Override
- public int hashCode() {
- final int prime = 31;
- int result = 1;
- result = prime * result + getId();
- result = prime * result + getQueryId().hashCode();
- return result;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (this == obj)
- return true;
- if (obj == null)
- return false;
- if (getClass() != obj.getClass())
- return false;
- SubQueryId other = (SubQueryId) obj;
- if (getId() != other.getId())
- return false;
- if (!getQueryId().equals(other.getQueryId()))
- return false;
- return true;
- }
-
- @Override
- public String toString() {
- StringBuilder builder = new StringBuilder(PREFIX);
- QueryId queryId = getQueryId();
- builder.append(QueryId.SEPARATOR).append(queryId.getApplicationId().getClusterTimestamp());
- builder.append(QueryId.SEPARATOR).append(
- QueryId.appIdFormat.get().format(queryId.getApplicationId().getId()));
- builder.append(QueryId.SEPARATOR).append(QueryId.attemptIdFormat.get().format(queryId.getAttemptId()))
- .append(QueryId.SEPARATOR)
- .append(subQueryIdFormat.get().format(getId()));
- return builder.toString();
- }
-
- @Override
- public int compareTo(SubQueryId other) {
- int queryIdComp = this.getQueryId().compareTo(other.getQueryId());
- if (queryIdComp == 0) {
- return this.getId() - other.getId();
- } else {
- return queryIdComp;
- }
- }
-
- public synchronized SubQueryIdProto getProto() {
- mergeLocalToProto();
- proto = viaProto ? proto : builder.build();
- viaProto = true;
- return proto;
- }
-
- private synchronized void mergeLocalToBuilder() {
- if (this.queryId != null
- && !this.queryId.getProto().equals(builder.getQueryId())) {
- builder.setQueryId(queryId.getProto());
- }
- }
-
- private synchronized void mergeLocalToProto() {
- if (viaProto)
- maybeInitBuilder();
- mergeLocalToBuilder();
- proto = builder.build();
- viaProto = true;
- }
-
- private synchronized void maybeInitBuilder() {
- if (viaProto || builder == null) {
- builder = SubQueryIdProto.newBuilder(proto);
- }
- viaProto = false;
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 21d0c63..2fe64e6 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -66,8 +66,7 @@ public class TajoConf extends YarnConfiguration {
// Service Addresses
TASKRUNNER_LISTENER_ADDRESS("tajo.master.taskrunnerlistener.addr", "0.0.0.0:0"), // used internally
CLIENT_SERVICE_ADDRESS("tajo.master.clientservice.addr", "127.0.0.1:9004"),
- QUERY_MASTER_MANAGER_SERVICE_ADDRESS("tajo.master.querymastermanager.addr", "127.0.0.1:9005"),
- QUERY_MASTER_CLIENT_SERVICE_ADDRESS("tajo.qmm.client.addr", "0.0.0.0:0"),
+ TAJO_MASTER_SERVICE_ADDRESS("tajo.master.manager.addr", "127.0.0.1:9005"),
//////////////////////////////////
// Catalog Configuration
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
index 7eaade8..05c0972 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
@@ -108,11 +108,11 @@ public class TUtil {
return list;
}
- public static QueryUnitAttemptId newQueryUnitAttemptId() {
+ public static QueryUnitAttemptId newQueryUnitAttemptId() {
return QueryIdFactory.newQueryUnitAttemptId(
QueryIdFactory.newQueryUnitId(
- QueryIdFactory.newSubQueryId(
- QueryIdFactory.newQueryId())), 0);
+ QueryIdFactory.newExecutionBlockId(
+ QueryIdFactory.newQueryId())), 0);
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
index 9dfbfbc..3b4bd51 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TajoIdUtils.java
@@ -18,104 +18,22 @@
package org.apache.tajo.util;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.BuilderUtils;
-import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryId;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.TajoIdProtos.SubQueryIdProto;
-import java.util.Iterator;
-
-import static org.apache.hadoop.yarn.util.StringHelper._split;
+import java.text.DecimalFormat;
public class TajoIdUtils {
- public static final String YARN_APPLICATION_PREFIX = "application";
- public static final String YARN_CONTAINER_PREFIX = "container";
- public static final String YARN_APPLICATION_ATTEMPT_PREFIX = "appattempt";
-
- /** It is mainly for DDL statements which don's have any query id. */
- public static final QueryId NullQueryId =
- TajoIdUtils.createQueryId(BuilderUtils.newApplicationId(0, 0), 0);
+ public static DecimalFormat MASTER_ID_FORMAT = new DecimalFormat("0000000000000");
- public static QueryId createQueryId(ApplicationId appId, int attemptId) {
- return newQueryId(appId, attemptId);
- }
-
- public static QueryId createQueryId(ApplicationAttemptId appAttemptId) {
- QueryId queryId = new QueryId();
- queryId.setApplicationId(appAttemptId.getApplicationId());
- queryId.setAttemptId(appAttemptId.getAttemptId());
- return queryId;
- }
-
- public static QueryId createQueryId(String queryId) {
- String[] split = queryId.split(QueryId.SEPARATOR);
- ApplicationId appId = BuilderUtils.newApplicationId(Long.valueOf(split[1]),
- Integer.parseInt(split[2]));
- int idInt = Integer.parseInt(split[3]);
- return newQueryId(appId, idInt);
- }
-
- public static SubQueryId createSubQueryId(QueryId queryId,
- int subQueryIdInt) {
- return newSubQueryId(queryId, subQueryIdInt);
- }
-
- public static QueryId newQueryId(ApplicationId appId, int id) {
- QueryId queryId = new QueryId();
- queryId.setApplicationId(appId);
- queryId.setAttemptId(id);
- return queryId;
- }
-
- public static SubQueryId newSubQueryId(QueryId jobId, int id) {
- SubQueryId taskId = new SubQueryId();
- taskId.setQueryId(jobId);
- taskId.setId(id);
- return taskId;
- }
-
- public static SubQueryId newSubQueryId(String subQueryId) {
- String [] split = subQueryId.split(QueryId.SEPARATOR);
- ApplicationId appId = BuilderUtils.newApplicationId(Long.valueOf(split[1]),
- Integer.valueOf(split[2]));
- QueryId queryId = TajoIdUtils.createQueryId(appId, Integer.valueOf(split[3]));
- return createSubQueryId(queryId, Integer.parseInt(split[4]));
- }
-
- public static SubQueryId newSubQueryId(SubQueryIdProto proto) {
- SubQueryId subId = new SubQueryId(proto);
- return subId;
- }
+ public static ExecutionBlockId createExecutionBlockId(String idStr) {
+ String[] tokens = idStr.split("_");
- public static ApplicationAttemptId toApplicationAttemptId(
- String applicationAttmeptIdStr) {
- //This methood from YARN.ConvertUtils
- Iterator<String> it = _split(applicationAttmeptIdStr).iterator();
- if (!it.next().equals(YARN_APPLICATION_ATTEMPT_PREFIX)) {
- throw new IllegalArgumentException("Invalid AppAttemptId prefix: "
- + applicationAttmeptIdStr);
- }
- try {
- return toApplicationAttemptId(it);
- } catch (NumberFormatException n) {
- throw new IllegalArgumentException("Invalid AppAttemptId: "
- + applicationAttmeptIdStr, n);
- }
+ return new ExecutionBlockId(new QueryId(tokens[1], Integer.parseInt(tokens[2])), Integer.parseInt(tokens[3]));
}
- private static ApplicationAttemptId toApplicationAttemptId(
- Iterator<String> it) throws NumberFormatException {
- //This methood from YARN.ConvertUtils
- ApplicationId appId = Records.newRecord(ApplicationId.class);
- appId.setClusterTimestamp(Long.parseLong(it.next()));
- appId.setId(Integer.parseInt(it.next()));
- ApplicationAttemptId appAttemptId = Records
- .newRecord(ApplicationAttemptId.class);
- appAttemptId.setApplicationId(appId);
- appAttemptId.setAttemptId(Integer.parseInt(it.next()));
- return appAttemptId;
+ public static QueryId parseQueryId(String idStr) {
+ String[] tokens = idStr.split("_");
+ return new QueryId(tokens[1], Integer.parseInt(tokens[2]));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/main/proto/TajoIdProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/proto/TajoIdProtos.proto b/tajo-common/src/main/proto/TajoIdProtos.proto
index 04c67f2..a87c825 100644
--- a/tajo-common/src/main/proto/TajoIdProtos.proto
+++ b/tajo-common/src/main/proto/TajoIdProtos.proto
@@ -21,15 +21,18 @@ option java_outer_classname = "TajoIdProtos";
option java_generic_services = false;
option java_generate_equals_and_hash = true;
-import "yarn_protos.proto";
+message QueryIdProto {
+ required string id = 1;
+ required int32 seq = 2;
+}
-message SubQueryIdProto {
- required ApplicationAttemptIdProto queryId = 1;
+message ExecutionBlockIdProto {
+ required QueryIdProto queryId = 1;
required int32 id = 2;
}
message QueryUnitIdProto {
- required SubQueryIdProto subQueryId = 1;
+ required ExecutionBlockIdProto executionBlockId = 1;
required int32 id = 2;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java b/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
index cbcaaf9..ba938b2 100644
--- a/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
+++ b/tajo-common/src/test/java/org/apache/tajo/datum/TestBitDatum.java
@@ -18,8 +18,8 @@
package org.apache.tajo.datum;
-import org.junit.Test;
import org.apache.tajo.common.TajoDataTypes.Type;
+import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/pom.xml b/tajo-core/tajo-core-backend/pom.xml
index ba5ad9f..6c3351f 100644
--- a/tajo-core/tajo-core-backend/pom.xml
+++ b/tajo-core/tajo-core-backend/pom.xml
@@ -127,8 +127,8 @@
<argument>src/main/proto/tajo_protos.proto</argument>
<argument>src/main/proto/ClientProtos.proto</argument>
<argument>src/main/proto/QueryMasterClientProtocol.proto</argument>
- <argument>src/main/proto/QueryMasterManagerProtocol.proto</argument>
- <argument>src/main/proto/QueryMasterProtocol.proto</argument>
+ <argument>src/main/proto/TajoMasterProtocol.proto</argument>
+ <argument>src/main/proto/TajoWorkerProtocol.proto</argument>
<argument>src/main/proto/TajoMasterClientProtocol.proto</argument>
</arguments>
</configuration>
@@ -329,7 +329,6 @@
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</dependency>
-
<dependency>
<groupId>jline</groupId>
<artifactId>jline</artifactId>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/log4j.properties
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/log4j.properties b/tajo-core/tajo-core-backend/src/main/java/log4j.properties
index c1ac487..749124c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/log4j.properties
+++ b/tajo-core/tajo-core-backend/src/main/java/log4j.properties
@@ -23,3 +23,6 @@ log4j.threshhold=ALL
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.conf=ERROR
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
index 2dd25de..55a97ca 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/cli/TajoCli.java
@@ -25,6 +25,7 @@ import jline.console.history.PersistentHistory;
import org.apache.commons.cli.*;
import org.apache.commons.lang.StringUtils;
import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.TajoProtos.QueryState;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.catalog.TableDesc;
@@ -34,7 +35,6 @@ import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.ipc.ClientProtos;
import org.apache.tajo.util.FileUtil;
-import org.apache.tajo.util.TajoIdUtils;
import java.io.File;
import java.io.InputStream;
@@ -264,6 +264,7 @@ public class TajoCli {
}
public int executeStatements(String line) throws Exception {
+
String stripped;
for (String statement : line.split(";")) {
stripped = StringUtils.chomp(statement);
@@ -282,12 +283,12 @@ public class TajoCli {
invokeCommand(cmds);
} else { // submit a query to TajoMaster
- ClientProtos.SubmitQueryResponse response = client.executeQuery(stripped);
+ ClientProtos.GetQueryStatusResponse response = client.executeQuery(stripped);
if (response.getResultCode() == ClientProtos.ResultCode.OK) {
QueryId queryId = null;
try {
queryId = new QueryId(response.getQueryId());
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
sout.println("OK");
} else {
getQueryResult(queryId);
@@ -298,9 +299,9 @@ public class TajoCli {
}
}
} else {
- if (response.hasErrorMessage()) {
- sout.println(response.getErrorMessage());
- }
+ if (response.hasErrorMessage()) {
+ sout.println(response.getErrorMessage());
+ }
}
}
}
@@ -313,7 +314,7 @@ public class TajoCli {
private void getQueryResult(QueryId queryId) {
// if query is empty string
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
return;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
index cd8706e..7e7b787 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/client/TajoClient.java
@@ -22,7 +22,9 @@ import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.TajoProtos.QueryState;
import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.TableDesc;
@@ -30,7 +32,6 @@ import org.apache.tajo.catalog.TableMeta;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.engine.query.ResultSetImpl;
-import org.apache.tajo.ipc.ClientProtos;
import org.apache.tajo.ipc.ClientProtos.*;
import org.apache.tajo.ipc.QueryMasterClientProtocol;
import org.apache.tajo.ipc.QueryMasterClientProtocol.QueryMasterClientProtocolService;
@@ -38,8 +39,6 @@ import org.apache.tajo.ipc.TajoMasterClientProtocol;
import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService;
import org.apache.tajo.rpc.ProtoBlockingRpcClient;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.StringProto;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.TajoIdUtils;
import java.io.IOException;
import java.net.InetSocketAddress;
@@ -128,7 +127,7 @@ public class TajoClient {
* In order to get the result, you should use {@link #getQueryResult(org.apache.tajo.QueryId)}
* or {@link #getQueryResultAndWait(org.apache.tajo.QueryId)}.
*/
- public ClientProtos.SubmitQueryResponse executeQuery(String tql) throws ServiceException {
+ public GetQueryStatusResponse executeQuery(String tql) throws ServiceException {
QueryRequest.Builder builder = QueryRequest.newBuilder();
builder.setQuery(tql);
@@ -147,9 +146,9 @@ public class TajoClient {
throws ServiceException, IOException {
QueryRequest.Builder builder = QueryRequest.newBuilder();
builder.setQuery(sql);
- SubmitQueryResponse response = tajoMasterService.submitQuery(null, builder.build());
+ GetQueryStatusResponse response = tajoMasterService.submitQuery(null, builder.build());
QueryId queryId = new QueryId(response.getQueryId());
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
return this.createNullResultSet(queryId);
} else {
return this.getQueryResultAndWait(queryId);
@@ -171,6 +170,9 @@ public class TajoClient {
String queryMasterHost = res.getQueryMasterHost();
if(queryMasterHost != null && !queryMasterHost.isEmpty()) {
connectionToQueryMaster(queryId, queryMasterHost, res.getQueryMasterPort());
+
+ QueryMasterClientProtocolService.BlockingInterface queryMasterService = queryMasterConnectionMap.get(queryId);
+ res = queryMasterService.getQueryStatus(null, builder.build());
}
}
return new QueryStatus(res);
@@ -204,7 +206,7 @@ public class TajoClient {
public ResultSet getQueryResult(QueryId queryId)
throws ServiceException, IOException {
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
return createNullResultSet(queryId);
}
@@ -214,14 +216,14 @@ public class TajoClient {
public ResultSet getQueryResultAndWait(QueryId queryId)
throws ServiceException, IOException {
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
return createNullResultSet(queryId);
}
QueryStatus status = getQueryStatus(queryId);
while(status != null && isQueryRunnning(status.getState())) {
try {
- Thread.sleep(1000);
+ Thread.sleep(500);
} catch (InterruptedException e) {
e.printStackTrace();
}
@@ -237,7 +239,7 @@ public class TajoClient {
}
} else {
- LOG.warn("Query " + status.getQueryId() + ") failed: " + status.getState());
+ LOG.warn("Query (" + status.getQueryId() + ") failed: " + status.getState());
//TODO throw SQLException(?)
return createNullResultSet(queryId);
@@ -249,7 +251,7 @@ public class TajoClient {
}
public TableDesc getResultDesc(QueryId queryId) throws ServiceException {
- if (queryId.equals(TajoIdUtils.NullQueryId)) {
+ if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) {
return null;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
index 7d430c5..cf10a4f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
@@ -19,7 +19,9 @@
package org.apache.tajo.engine.query;
import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.ipc.QueryMasterProtocol.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Fetch;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProtoOrBuilder;
import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
import org.apache.tajo.storage.Fragment;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
index 808b910..8cbe956 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetImpl.java
@@ -16,9 +16,6 @@
* limitations under the License.
*/
-/**
- *
- */
package org.apache.tajo.engine.query;
import com.google.common.collect.Lists;
@@ -297,7 +294,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Array getArray(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -308,7 +304,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Array getArray(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -319,7 +314,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public InputStream getAsciiStream(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -330,7 +324,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public InputStream getAsciiStream(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -363,7 +356,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public BigDecimal getBigDecimal(int arg0, int arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -374,7 +366,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public BigDecimal getBigDecimal(String arg0, int arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -385,7 +376,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public InputStream getBinaryStream(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -396,7 +386,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public InputStream getBinaryStream(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -499,7 +488,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Reader getCharacterStream(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -510,7 +498,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Reader getCharacterStream(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -561,7 +548,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Date getDate(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -572,7 +558,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Date getDate(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -583,7 +568,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Date getDate(int arg0, Calendar arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -594,7 +578,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Date getDate(String arg0, Calendar arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -741,7 +724,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Reader getNCharacterStream(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -752,7 +734,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Reader getNCharacterStream(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -783,7 +764,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public String getNString(int fieldId) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -794,7 +774,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public String getNString(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -832,7 +811,6 @@ public class ResultSetImpl implements ResultSet {
@Override
public Object getObject(int arg0, Map<String, Class<?>> arg1)
throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -844,7 +822,6 @@ public class ResultSetImpl implements ResultSet {
@Override
public Object getObject(String arg0, Map<String, Class<?>> arg1)
throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -855,7 +832,6 @@ public class ResultSetImpl implements ResultSet {
*/
public <T> T getObject(String arg0, Class<T> arg1)
throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -866,10 +842,9 @@ public class ResultSetImpl implements ResultSet {
*/
public <T> T getObject(int arg0, Class<T> arg1)
throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
-
+
/*
* (non-Javadoc)
*
@@ -907,7 +882,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public RowId getRowId(int fieldId) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -918,7 +892,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public RowId getRowId(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1007,7 +980,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Time getTime(int fieldId) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1018,7 +990,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Time getTime(String name) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1029,7 +1000,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Time getTime(int fieldId, Calendar arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1040,7 +1010,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Time getTime(String name, Calendar arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1051,7 +1020,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Timestamp getTimestamp(int fieldId) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1062,7 +1030,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Timestamp getTimestamp(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1073,7 +1040,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Timestamp getTimestamp(int fieldId, Calendar arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1084,7 +1050,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public Timestamp getTimestamp(String arg0, Calendar arg1) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1105,7 +1070,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public URL getURL(int fieldId) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1116,7 +1080,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public URL getURL(String arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
@@ -1147,7 +1110,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public SQLWarning getWarnings() throws SQLException {
- // TODO
throw new UnsupportedException();
}
@@ -1298,7 +1260,6 @@ public class ResultSetImpl implements ResultSet {
*/
@Override
public boolean relative(int arg0) throws SQLException {
- // TODO Auto-generated method stub
throw new UnsupportedException();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
index bb4008f..c6fc632 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
@@ -23,13 +23,13 @@ package org.apache.tajo.ipc.protocolrecords;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.common.ProtoObject;
-import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
import org.apache.tajo.storage.Fragment;
import java.net.URI;
import java.util.List;
-public interface QueryUnitRequest extends ProtoObject<QueryMasterProtocol.QueryUnitRequestProto> {
+public interface QueryUnitRequest extends ProtoObject<TajoWorkerProtocol.QueryUnitRequestProto> {
public QueryUnitAttemptId getId();
public List<Fragment> getFragments();
@@ -39,7 +39,7 @@ public interface QueryUnitRequest extends ProtoObject<QueryMasterProtocol.QueryU
public boolean isInterQuery();
public void setInterQuery();
public void addFetch(String name, URI uri);
- public List<QueryMasterProtocol.Fetch> getFetches();
+ public List<TajoWorkerProtocol.Fetch> getFetches();
public boolean shouldDie();
public void setShouldDie();
}
[4/8] TAJO-127: Implement Tajo Resource Manager. (hyoungjunkim via
hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
new file mode 100644
index 0000000..0c1d056
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -0,0 +1,445 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.Clock;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.logical.LogicalNode;
+import org.apache.tajo.engine.planner.logical.LogicalRootNode;
+import org.apache.tajo.engine.planner.logical.NodeType;
+import org.apache.tajo.engine.planner.logical.ScanNode;
+import org.apache.tajo.master.TajoAsyncDispatcher;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.worker.AbstractResourceAllocator;
+import org.apache.tajo.worker.TajoResourceAllocator;
+import org.apache.tajo.worker.YarnResourceAllocator;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QueryMasterTask extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
+
+ // query submission directory is private!
+ final public static FsPermission USER_DIR_PERMISSION =
+ FsPermission.createImmutable((short) 0700); // rwx--------
+
+ private QueryId queryId;
+
+ private QueryContext queryContext;
+
+ private QueryMaster.QueryMasterContext queryMasterContext;
+
+ private Query query;
+
+ private MasterPlan masterPlan;
+
+ private String logicalPlanJson;
+
+ private TajoAsyncDispatcher dispatcher;
+
+ private final long querySubmitTime;
+
+ private boolean isCreateTableStmt;
+
+ private Path outputPath;
+
+ private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+
+ private QueryConf queryConf;
+
+ private AtomicLong lastClientHeartbeat = new AtomicLong(-1);
+
+ private AbstractResourceAllocator resourceAllocator;
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
+ QueryId queryId, String logicalPlanJson) {
+ super(QueryMasterTask.class.getName());
+ this.queryMasterContext = queryMasterContext;
+ this.queryId = queryId;
+ this.logicalPlanJson = logicalPlanJson;
+ this.querySubmitTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ queryConf = new QueryConf(conf);
+ queryConf.addResource(new Path(QueryConf.QUERY_MASTER_FILENAME));
+ try {
+ queryContext = new QueryContext();
+ String resourceManagerClassName = conf.get("tajo.resource.manager",
+ TajoWorkerResourceManager.class.getCanonicalName());
+
+ if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+ resourceAllocator = new TajoResourceAllocator(queryContext);
+ } else {
+ resourceAllocator = new YarnResourceAllocator(queryContext);
+ }
+ addService(resourceAllocator);
+
+ dispatcher = new TajoAsyncDispatcher(queryId.toString());
+ addService(dispatcher);
+
+ dispatcher.register(SubQueryEventType.class, new SubQueryEventDispatcher());
+ dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
+ dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+ dispatcher.register(QueryFinishEvent.EventType.class, new QueryFinishEventHandler());
+ dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+
+ initStagingDir();
+
+ super.init(queryConf);
+ } catch (IOException e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ public boolean isStopped() {
+ return stopped.get();
+ }
+
+ @Override
+ public void start() {
+ startQuery();
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(stopped.get()) {
+ return;
+ }
+ stopped.set(true);
+
+ LOG.info("====> Stopping QueryMasterTask:" + queryId);
+
+ queryMasterContext.getWorkerContext().getTajoMasterRpcClient()
+ .stopQueryMaster(null, queryId.getProto(), NullCallback.get());
+
+ super.stop();
+
+ LOG.info("====> Stopped QueryMasterTask:" + queryId);
+ }
+
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+ ExecutionBlockId id = event.getExecutionBlockId();
+ query.getSubQuery(id).handleTaskRequestEvent(event);
+ }
+
+ private class SubQueryEventDispatcher implements EventHandler<SubQueryEvent> {
+ public void handle(SubQueryEvent event) {
+ ExecutionBlockId id = event.getSubQueryId();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("SubQueryEventDispatcher:" + id + "," + event.getType());
+ }
+ //Query query = queryMasterTasks.get(id.getQueryId()).getQuery();
+ query.getSubQuery(id).handle(event);
+ }
+ }
+
+ private class TaskEventDispatcher
+ implements EventHandler<TaskEvent> {
+ public void handle(TaskEvent event) {
+ QueryUnitId taskId = event.getTaskId();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
+ }
+ //Query query = queryMasterTasks.get(taskId.getExecutionBlockId().getQueryId()).getQuery();
+ QueryUnit task = query.getSubQuery(taskId.getExecutionBlockId()).
+ getQueryUnit(taskId);
+ task.handle(event);
+ }
+ }
+
+ private class TaskAttemptEventDispatcher
+ implements EventHandler<TaskAttemptEvent> {
+ public void handle(TaskAttemptEvent event) {
+ QueryUnitAttemptId attemptId = event.getTaskAttemptId();
+ //Query query = queryMasterTasks.get(attemptId.getQueryUnitId().getExecutionBlockId().getQueryId()).getQuery();
+ SubQuery subQuery = query.getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId());
+ QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
+ QueryUnitAttempt attempt = task.getAttempt(attemptId);
+ attempt.handle(event);
+ }
+ }
+
+ private class TaskSchedulerDispatcher
+ implements EventHandler<TaskSchedulerEvent> {
+ public void handle(TaskSchedulerEvent event) {
+ //Query query = queryMasterTasks.get(event.getExecutionBlockId().getQueryId()).getQuery();
+ SubQuery subQuery = query.getSubQuery(event.getExecutionBlockId());
+ subQuery.getTaskScheduler().handle(event);
+ }
+ }
+
+ private class QueryFinishEventHandler implements EventHandler<QueryFinishEvent> {
+ @Override
+ public void handle(QueryFinishEvent event) {
+ QueryId queryId = event.getQueryId();
+ LOG.info("Query end notification started for QueryId : " + queryId);
+ //QueryMaster must be lived until client fetching all query result data.
+ }
+ }
+
+ public synchronized void startQuery() {
+ LOG.info("Plan JSON:" + logicalPlanJson);
+ if(query != null) {
+ LOG.warn("Query already started");
+ return;
+ }
+
+ try {
+ LogicalRootNode logicalNodeRoot = (LogicalRootNode) CoreGsonHelper.fromJson(logicalPlanJson, LogicalNode.class);
+ LogicalNode[] scanNodes = PlannerUtil.findAllNodes(logicalNodeRoot, NodeType.SCAN);
+ if(scanNodes != null) {
+ for(LogicalNode eachScanNode: scanNodes) {
+ ScanNode scanNode = (ScanNode)eachScanNode;
+ tableDescMap.put(scanNode.getFromTable().getTableName(), scanNode.getFromTable().getTableDesc());
+ }
+ }
+ MasterPlan globalPlan = queryMasterContext.getGlobalPlanner().build(queryId, logicalNodeRoot);
+ this.masterPlan = queryMasterContext.getGlobalOptimizer().optimize(globalPlan);
+
+ query = new Query(queryContext, queryId, querySubmitTime,
+ "", queryContext.getEventHandler(), masterPlan);
+
+ dispatcher.register(QueryEventType.class, query);
+
+ queryContext.getEventHandler().handle(new QueryEvent(queryId,
+ QueryEventType.INIT));
+ queryContext.getEventHandler().handle(new QueryEvent(queryId,
+ QueryEventType.START));
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ //TODO how set query failed(???)
+ //send FAIL query status
+ //this.statusMessage = StringUtils.stringifyException(e);
+ }
+ }
+
+ /**
+ * It initializes the final output and staging directory and sets
+ * them to variables.
+ */
+ private void initStagingDir() throws IOException {
+ String realUser;
+ String currentUser;
+ UserGroupInformation ugi;
+ ugi = UserGroupInformation.getLoginUser();
+ realUser = ugi.getShortUserName();
+ currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+ String givenOutputTableName = queryConf.getOutputTable();
+ Path stagingDir;
+
+ // If final output directory is not given by an user,
+ // we use the query id as a output directory.
+ if (givenOutputTableName == null || givenOutputTableName.isEmpty()) {
+ this.isCreateTableStmt = false;
+ FileSystem defaultFS = FileSystem.get(queryConf);
+
+ Path homeDirectory = defaultFS.getHomeDirectory();
+ if (!defaultFS.exists(homeDirectory)) {
+ defaultFS.mkdirs(homeDirectory, new FsPermission(USER_DIR_PERMISSION));
+ }
+
+ Path userQueryDir = new Path(homeDirectory, TajoConstants.USER_QUERYDIR_PREFIX);
+
+ if (defaultFS.exists(userQueryDir)) {
+ FileStatus fsStatus = defaultFS.getFileStatus(userQueryDir);
+ String owner = fsStatus.getOwner();
+
+ if (!(owner.equals(currentUser) || owner.equals(realUser))) {
+ throw new IOException("The ownership on the user's query " +
+ "directory " + userQueryDir + " is not as expected. " +
+ "It is owned by " + owner + ". The directory must " +
+ "be owned by the submitter " + currentUser + " or " +
+ "by " + realUser);
+ }
+
+ if (!fsStatus.getPermission().equals(USER_DIR_PERMISSION)) {
+ LOG.info("Permissions on staging directory " + userQueryDir + " are " +
+ "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
+ "to correct value " + USER_DIR_PERMISSION);
+ defaultFS.setPermission(userQueryDir, new FsPermission(USER_DIR_PERMISSION));
+ }
+ } else {
+ defaultFS.mkdirs(userQueryDir,
+ new FsPermission(USER_DIR_PERMISSION));
+ }
+
+ stagingDir = StorageUtil.concatPath(userQueryDir, queryId.toString());
+
+ if (defaultFS.exists(stagingDir)) {
+ throw new IOException("The staging directory " + stagingDir
+ + "already exists. The directory must be unique to each query");
+ } else {
+ defaultFS.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
+ }
+
+ // Set the query id to the output table name
+ queryConf.setOutputTable(queryId.toString());
+
+ } else {
+ this.isCreateTableStmt = true;
+ Path warehouseDir = new Path(queryConf.getVar(TajoConf.ConfVars.ROOT_DIR),
+ TajoConstants.WAREHOUSE_DIR);
+ stagingDir = new Path(warehouseDir, queryConf.getOutputTable());
+
+ FileSystem fs = warehouseDir.getFileSystem(queryConf);
+ if (fs.exists(stagingDir)) {
+ throw new IOException("The staging directory " + stagingDir
+ + " already exists. The directory must be unique to each query");
+ } else {
+ // TODO - should have appropriate permission
+ fs.mkdirs(stagingDir, new FsPermission(USER_DIR_PERMISSION));
+ }
+ }
+
+ queryConf.setOutputPath(stagingDir);
+ outputPath = stagingDir;
+ LOG.info("Initialized Query Staging Dir: " + outputPath);
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public void expiredSessionTimeout() {
+ stop();
+ }
+
+ public QueryContext getQueryContext() {
+ return queryContext;
+ }
+
+ public EventHandler getEventHandler() {
+ return queryContext.getEventHandler();
+ }
+
+ public void touchSessionTime() {
+ this.lastClientHeartbeat.set(System.currentTimeMillis());
+ }
+
+ public long getLastClientHeartbeat() {
+ return this.lastClientHeartbeat.get();
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public TajoProtos.QueryState getState() {
+ if(query == null) {
+ return TajoProtos.QueryState.QUERY_NOT_ASSIGNED;
+ } else {
+ return query.getState();
+ }
+ }
+
+ public class QueryContext {
+ EventHandler eventHandler;
+ public QueryMaster.QueryMasterContext getQueryMasterContext() {
+ return queryMasterContext;
+ }
+
+ public QueryConf getConf() {
+ return queryConf;
+ }
+
+ public Clock getClock() {
+ return queryMasterContext.getClock();
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public StorageManager getStorageManager() {
+ return queryMasterContext.getStorageManager();
+ }
+
+ public Path getOutputPath() {
+ return outputPath;
+ }
+
+ public boolean isCreateTableQuery() {
+ return isCreateTableStmt;
+ }
+
+ public synchronized EventHandler getEventHandler() {
+ if(eventHandler == null) {
+ eventHandler = dispatcher.getEventHandler();
+ }
+ return eventHandler;
+ }
+
+ public TajoAsyncDispatcher getDispatcher() {
+ return dispatcher;
+ }
+
+ public SubQuery getSubQuery(ExecutionBlockId id) {
+ return query.getSubQuery(id);
+ }
+
+ public Map<String, TableDesc> getTableDescMap() {
+ return tableDescMap;
+ }
+
+ public float getProgress() {
+ if(query == null) {
+ return 0.0f;
+ }
+ return query.getProgress();
+ }
+
+ public AbstractResourceAllocator getResourceAllocator() {
+ return (AbstractResourceAllocator)resourceAllocator;
+ }
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
index 69143c9..ec54244 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
@@ -30,11 +30,12 @@ import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.QueryUnitId;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.ipc.QueryMasterProtocol.Partition;
import org.apache.tajo.engine.planner.logical.*;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Partition;
import org.apache.tajo.master.TaskState;
import org.apache.tajo.master.event.*;
import org.apache.tajo.storage.Fragment;
+import org.apache.tajo.util.TajoIdUtils;
import java.net.URI;
import java.net.URISyntaxException;
@@ -299,9 +300,8 @@ public class QueryUnit implements EventHandler<TaskEvent> {
}
public QueryUnitAttempt newAttempt() {
- QueryUnitAttempt attempt = new QueryUnitAttempt(
- QueryIdFactory.newQueryUnitAttemptId(this.getId(),
- ++lastAttemptId), this, eventHandler);
+ QueryUnitAttempt attempt = new QueryUnitAttempt(QueryIdFactory.newQueryUnitAttemptId(
+ this.getId(), ++lastAttemptId), this, eventHandler);
return attempt;
}
@@ -310,7 +310,7 @@ public class QueryUnit implements EventHandler<TaskEvent> {
}
public QueryUnitAttempt getAttempt(int attempt) {
- return this.attempts.get(new QueryUnitAttemptId(this.getId(), attempt));
+ return this.attempts.get(QueryIdFactory.newQueryUnitAttemptId(this.getId(), attempt));
}
public QueryUnitAttempt getLastAttempt() {
@@ -433,7 +433,7 @@ public class QueryUnit implements EventHandler<TaskEvent> {
stateMachine.doTransition(event.getType(), event);
} catch (InvalidStateTransitonException e) {
LOG.error("Can't handle this event at current state", e);
- eventHandler.handle(new QueryEvent(getId().getQueryId(),
+ eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
QueryEventType.INTERNAL_ERROR));
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
index add42aa..9b9c63f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnitAttempt.java
@@ -26,11 +26,12 @@ import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.TajoProtos.TaskAttemptState;
import org.apache.tajo.catalog.statistics.TableStat;
-import org.apache.tajo.ipc.QueryMasterProtocol.Partition;
-import org.apache.tajo.ipc.QueryMasterProtocol.TaskCompletionReport;
-import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.ipc.TajoWorkerProtocol.Partition;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
import org.apache.tajo.master.event.*;
import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.util.TajoIdUtils;
import java.util.*;
import java.util.concurrent.locks.Lock;
@@ -146,6 +147,10 @@ public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
return this.hostName;
}
+ public int getPort() {
+ return this.port;
+ }
+
public void setHost(String host) {
this.hostName = host;
}
@@ -324,7 +329,7 @@ public class QueryUnitAttempt implements EventHandler<TaskAttemptEvent> {
} catch (InvalidStateTransitonException e) {
LOG.error("Can't handle this event at current state of "
+ event.getTaskAttemptId() + ")", e);
- eventHandler.handle(new QueryEvent(getId().getQueryId(),
+ eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
QueryEventType.INTERNAL_ERROR));
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
index cccfcba..a9b0e4b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -21,8 +21,8 @@ package org.apache.tajo.master.querymaster;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
import org.apache.tajo.catalog.statistics.TableStat;
@@ -70,7 +70,7 @@ public class Repartitioner {
for (int i =0; i < 2; i++) {
// TODO - temporarily tables should be stored in temporarily catalog for each query
TableDesc tableDesc = subQuery.getContext().getTableDescMap().get(scans[i].getFromTable().getTableName());
- if (scans[i].getTableId().startsWith(SubQueryId.PREFIX)) {
+ if (scans[i].getTableId().startsWith(ExecutionBlockId.EB_ID_PREFIX)) {
tablePath = subQuery.getStorageManager().getTablePath(scans[i].getTableId());
} else {
tablePath = tableDesc.getPath();
@@ -355,12 +355,13 @@ public class Repartitioner {
}
public static String createBasicFetchUri(String hostName, int port,
- SubQueryId childSid,
+ ExecutionBlockId childSid,
int taskId, int attemptId) {
String scheme = "http://";
StringBuilder sb = new StringBuilder(scheme);
- sb.append(hostName).append(":").append(port)
- .append("/?").append("sid=").append(childSid.getId())
+ sb.append(hostName).append(":").append(port).append("/?")
+ .append("qid=").append(childSid.getQueryId().toString())
+ .append("&sid=").append(childSid.getId())
.append("&").append("ta=").append(taskId).append("_").append(attemptId)
.append("&").append("p=0")
.append("&").append("type=r");
@@ -436,15 +437,16 @@ public class Repartitioner {
return tasks;
}
- public static Collection<URI> createHashFetchURL(String hostAndPort, SubQueryId childSid,
+ public static Collection<URI> createHashFetchURL(String hostAndPort, ExecutionBlockId ebid,
int partitionId, PartitionType type,
List<IntermediateEntry> entries) {
String scheme = "http://";
StringBuilder urlPrefix = new StringBuilder(scheme);
- urlPrefix.append(hostAndPort)
- .append("/?").append("sid=").append(childSid.getId())
- .append("&").append("p=").append(partitionId)
- .append("&").append("type=");
+ urlPrefix.append(hostAndPort).append("/?")
+ .append("qid=").append(ebid.getQueryId().toString())
+ .append("&sid=").append(ebid.getId())
+ .append("&p=").append(partitionId)
+ .append("&type=");
if (type == PartitionType.HASH) {
urlPrefix.append("h");
} else if (type == PartitionType.RANGE) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 94a6af8..fc60df7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -31,9 +31,9 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.*;
import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.QueryUnitId;
-import org.apache.tajo.SubQueryId;
import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.TableDesc;
import org.apache.tajo.catalog.TableMeta;
@@ -52,7 +52,6 @@ import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
import org.apache.tajo.master.TaskScheduler;
import org.apache.tajo.master.TaskSchedulerImpl;
import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
import org.apache.tajo.storage.Fragment;
import org.apache.tajo.storage.StorageManager;
@@ -79,7 +78,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
private EventHandler eventHandler;
private final StorageManager sm;
private TaskSchedulerImpl taskScheduler;
- private QueryContext context;
+ private QueryMasterTask.QueryContext context;
private long startTime;
private long finishTime;
@@ -87,7 +86,6 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
volatile Map<QueryUnitId, QueryUnit> tasks = new ConcurrentHashMap<QueryUnitId, QueryUnit>();
volatile Map<ContainerId, Container> containers = new ConcurrentHashMap<ContainerId, Container>();
-
private static ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition();
private StateMachine<SubQueryState, SubQueryEventType, SubQueryEvent>
stateMachine;
@@ -140,7 +138,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
private int completedTaskCount = 0;
- public SubQuery(QueryContext context, ExecutionBlock block, StorageManager sm) {
+ public SubQuery(QueryMasterTask.QueryContext context, ExecutionBlock block, StorageManager sm) {
this.context = context;
this.block = block;
this.sm = sm;
@@ -152,7 +150,12 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
stateMachine = stateMachineFactory.make(this);
}
- public QueryContext getContext() {
+ public static boolean isRunningState(SubQueryState state) {
+ return state == SubQueryState.INIT || state == SubQueryState.NEW ||
+ state == SubQueryState.CONTAINER_ALLOCATED || state == SubQueryState.RUNNING;
+ }
+
+ public QueryMasterTask.QueryContext getContext() {
return context;
}
@@ -238,7 +241,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return context.getSubQuery(block.getChildBlock(scanForChild).getId());
}
- public SubQueryId getId() {
+ public ExecutionBlockId getId() {
return block.getId();
}
@@ -399,7 +402,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
@Override
public void handle(SubQueryEvent event) {
if (LOG.isDebugEnabled()) {
- LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType());
+ LOG.debug("Processing " + event.getSubQueryId() + " of type " + event.getType() + ", preState=" + getState());
}
try {
@@ -420,13 +423,15 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
+ getState());
}
}
- }
-
- finally {
+ } finally {
writeLock.unlock();
}
}
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+ taskScheduler.handleTaskRequestEvent(event);
+ }
+
private static class InitAndRequestContainer implements MultipleArcTransition<SubQuery,
SubQueryEvent, SubQueryState> {
@@ -564,7 +569,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
} else { // Case 3: Others (Sort or Aggregation)
int numTasks = getNonLeafTaskNum(subQuery);
- SubQueryId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
+ ExecutionBlockId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
SubQuery child = subQuery.context.getSubQuery(childId);
tasks = Repartitioner.createNonLeafTask(subQuery, child, numTasks);
}
@@ -594,7 +599,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return maxTaskNum;
}
- public static long getInputVolume(QueryContext context, ExecutionBlock execBlock) {
+ public static long getInputVolume(QueryMasterTask.QueryContext context, ExecutionBlock execBlock) {
Map<String, TableDesc> tableMap = context.getTableDescMap();
if (execBlock.isLeafBlock()) {
ScanNode outerScan = execBlock.getScanNodes()[0];
@@ -615,14 +620,13 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
ExecutionBlock execBlock = subQuery.getBlock();
QueryUnit [] tasks = subQuery.getQueryUnits();
- int numClusterNodes = subQuery.getContext().getNumClusterNode();
- TajoConf conf = subQuery.getContext().getConf();
- int workerNum = conf.getIntVar(ConfVars.MAX_WORKER_PER_NODE);
- int numRequest = Math.min(tasks.length, numClusterNodes * workerNum);
+ //TODO refresh worker's numClusterNodes
+ int numClusterNodes = subQuery.getContext().getResourceAllocator().getNumClusterNode();
+ int numRequest = numClusterNodes == 0 ? tasks.length: Math.min(tasks.length, numClusterNodes * 4);
final Resource resource = Records.newRecord(Resource.class);
- // TODO - for each different subquery, the volume of resource should be different.
- resource.setMemory(2000);
+
+ resource.setMemory(512);
Priority priority = Records.newRecord(Priority.class);
priority.setPriority(subQuery.getPriority());
@@ -698,8 +702,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_LAUNCH,
subQuery.getId(), allocationEvent.getAllocatedContainer()));
- subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
- SubQueryEventType.SQ_START));
+ subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(), SubQueryEventType.SQ_START));
}
}
@@ -734,7 +737,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
QueryUnitAttempt task = subQuery.getQueryUnit(taskEvent.getTaskId()).getSuccessfulAttempt();
LOG.info(subQuery.getId() + " SubQuery Succeeded " + subQuery.completedTaskCount + "/"
- + subQuery.tasks.size() + " on " + task.getHost());
+ + subQuery.tasks.size() + " on " + task.getHost() + ":" + task.getPort());
if (subQuery.completedTaskCount == subQuery.tasks.size()) {
subQuery.eventHandler.handle(new SubQueryEvent(subQuery.getId(),
SubQueryEventType.SQ_SUBQUERY_COMPLETED));
@@ -750,6 +753,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
// TODO - Commit subQuery & do cleanup
// TODO - records succeeded, failed, killed completed task
// TODO - records metrics
+ LOG.info("SubQuery finished:" + subQuery.getId());
subQuery.stopScheduler();
subQuery.releaseContainers();
subQuery.finish();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java
deleted file mode 100644
index c02ddaf..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/RMContainerAllocator.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.master.rm;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnException;
-import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.records.AMResponse;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.client.AMRMClientImpl;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.tajo.SubQueryId;
-import org.apache.tajo.master.event.ContainerAllocationEvent;
-import org.apache.tajo.master.event.ContainerAllocatorEventType;
-import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
-import org.apache.tajo.master.querymaster.QueryMaster.QueryContext;
-import org.apache.tajo.master.querymaster.SubQueryState;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class RMContainerAllocator extends AMRMClientImpl
- implements EventHandler<ContainerAllocationEvent> {
-
- /** Class Logger */
- private static final Log LOG = LogFactory.getLog(RMContainerAllocator.
- class.getName());
-
- private QueryContext context;
- private final EventHandler eventHandler;
-
- public RMContainerAllocator(QueryContext context) {
- super(context.getApplicationAttemptId());
- this.context = context;
- this.eventHandler = context.getDispatcher().getEventHandler();
- }
-
- public void init(Configuration conf) {
- super.init(conf);
- }
-
- private static final int WAIT_INTERVAL_AVAILABLE_NODES = 500; // 0.5 second
- public void start() {
- super.start();
-
- RegisterApplicationMasterResponse response;
- try {
- response = registerApplicationMaster("locahost", 10080, "http://localhost:1234");
- context.setMaxContainerCapability(response.getMaximumResourceCapability().getMemory());
- context.setMinContainerCapability(response.getMinimumResourceCapability().getMemory());
-
- // If the number of cluster nodes is ZERO, it waits for available nodes.
- AllocateResponse allocateResponse = allocate(0.0f);
- while(allocateResponse.getNumClusterNodes() < 1) {
- try {
- Thread.sleep(WAIT_INTERVAL_AVAILABLE_NODES);
- LOG.info("Waiting for Available Cluster Nodes");
- allocateResponse = allocate(0);
- } catch (InterruptedException e) {
- LOG.error(e);
- }
- }
- context.setNumClusterNodes(allocateResponse.getNumClusterNodes());
- } catch (YarnRemoteException e) {
- LOG.error(e);
- }
-
- startAllocatorThread();
- }
-
- protected Thread allocatorThread;
- private final AtomicBoolean stopped = new AtomicBoolean(false);
- private int rmPollInterval = 100;//millis
- protected void startAllocatorThread() {
- allocatorThread = new Thread(new Runnable() {
- @Override
- public void run() {
- while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
- try {
- try {
- heartbeat();
- } catch (YarnException e) {
- LOG.error("Error communicating with RM: " + e.getMessage() , e);
- return;
- } catch (Exception e) {
- LOG.error("ERROR IN CONTACTING RM. ", e);
- // TODO: for other exceptions
- }
- Thread.sleep(rmPollInterval);
- } catch (InterruptedException e) {
- if (!stopped.get()) {
- LOG.warn("Allocated thread interrupted. Returning.");
- }
- break;
- }
- }
- LOG.info("Allocated thread stopped");
- }
- });
- allocatorThread.setName("RMContainerAllocator");
- allocatorThread.start();
- }
-
- public void stop() {
- stopped.set(true);
- allocatorThread.interrupt();
- LOG.info("RMContainerAllocator stopped");
- super.stop();
- }
-
- private final Map<Priority, SubQueryId> subQueryMap =
- new HashMap<Priority, SubQueryId>();
- private AtomicLong prevReportTime = new AtomicLong(0);
- private int reportInterval = 5 * 1000; // second
-
- public void heartbeat() throws Exception {
- AllocateResponse allocateResponse = allocate(context.getProgress());
- AMResponse response = allocateResponse.getAMResponse();
- List<Container> allocatedContainers = response.getAllocatedContainers();
-
- long currentTime = System.currentTimeMillis();
- if((currentTime - prevReportTime.longValue()) >= reportInterval){
- LOG.debug("Available Cluster Nodes: " + allocateResponse.getNumClusterNodes());
- LOG.debug("Num of Allocated Containers: " + allocatedContainers.size());
- LOG.info("Available Resource: " + response.getAvailableResources());
- prevReportTime.set(currentTime);
- }
-
- if (allocatedContainers.size() > 0) {
- LOG.info("================================================================");
- for (Container container : response.getAllocatedContainers()) {
- LOG.info("> Container Id: " + container.getId());
- LOG.info("> Node Id: " + container.getNodeId());
- LOG.info("> Resource (Mem): " + container.getResource().getMemory());
- LOG.info("> State : " + container.getState());
- LOG.info("> Priority: " + container.getPriority());
- }
- LOG.info("================================================================");
-
- Map<SubQueryId, List<Container>> allocated = new HashMap<SubQueryId, List<Container>>();
-
- for (Container container : allocatedContainers) {
- SubQueryId subQueryId = subQueryMap.get(container.getPriority());
- SubQueryState state = context.getSubQuery(subQueryId).getState();
- if (!(isRunningState(state))) {
- releaseAssignedContainer(container.getId());
- } else {
- if (allocated.containsKey(subQueryId)) {
- allocated.get(subQueryId).add(container);
- } else {
- allocated.put(subQueryId, Lists.newArrayList(container));
- }
- }
- }
-
- for (Entry<SubQueryId, List<Container>> entry : allocated.entrySet()) {
- eventHandler.handle(new SubQueryContainerAllocationEvent(entry.getKey(), entry.getValue()));
- }
- }
- }
-
- private static boolean isRunningState(SubQueryState state) {
- return state == SubQueryState.INIT || state == SubQueryState.NEW ||
- state == SubQueryState.CONTAINER_ALLOCATED || state == SubQueryState.RUNNING;
- }
-
- @Override
- public void handle(ContainerAllocationEvent event) {
-
- if (event.getType() == ContainerAllocatorEventType.CONTAINER_REQ) {
- LOG.info(event);
- subQueryMap.put(event.getPriority(), event.getSubQueryId());
- addContainerRequest(new ContainerRequest(event.getCapability(), null, null,
- event.getPriority(), event.getRequiredNum()));
-
- } else if (event.getType() == ContainerAllocatorEventType.CONTAINER_DEALLOCATE) {
- LOG.info(event);
- } else {
- LOG.info(event);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
new file mode 100644
index 0000000..9f7aeb1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainer.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import org.apache.hadoop.yarn.api.records.*;
+
+public class TajoWorkerContainer implements Container {
+ ContainerId id;
+ NodeId nodeId;
+ WorkerResource workerResource;
+
+ public WorkerResource getWorkerResource() {
+ return workerResource;
+ }
+
+ public void setWorkerResource(WorkerResource workerResource) {
+ this.workerResource = workerResource;
+ }
+
+ @Override
+ public ContainerId getId() {
+ return id;
+ }
+
+ @Override
+ public void setId(ContainerId id) {
+ this.id = id;
+ }
+
+ @Override
+ public NodeId getNodeId() {
+ return nodeId;
+ }
+
+ @Override
+ public void setNodeId(NodeId nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public String getNodeHttpAddress() {
+ return null; //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void setNodeHttpAddress(String nodeHttpAddress) {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public Resource getResource() {
+ return null; //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void setResource(Resource resource) {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public Priority getPriority() {
+ return null; //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void setPriority(Priority priority) {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public ContainerState getState() {
+ return null; //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void setState(ContainerState state) {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public ContainerToken getContainerToken() {
+ return null; //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void setContainerToken(ContainerToken containerToken) {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public ContainerStatus getContainerStatus() {
+ return null; //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public void setContainerStatus(ContainerStatus containerStatus) {
+ //To change body of implemented methods use File | Settings | File Templates.
+ }
+
+ @Override
+ public int compareTo(Container container) {
+ return 0; //To change body of implemented methods use File | Settings | File Templates.
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
new file mode 100644
index 0000000..f104637
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerContainerId.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+
+public class TajoWorkerContainerId extends ContainerId {
+ ApplicationAttemptId applicationAttemptId;
+ int id;
+
+ @Override
+ public ApplicationAttemptId getApplicationAttemptId() {
+ return applicationAttemptId;
+ }
+
+ @Override
+ public void setApplicationAttemptId(ApplicationAttemptId atId) {
+ this.applicationAttemptId = atId;
+ }
+
+ @Override
+ public int getId() {
+ return id;
+ }
+
+ @Override
+ public void setId(int id) {
+ this.id = id;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
new file mode 100644
index 0000000..be4b800
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -0,0 +1,394 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.master.querymaster.QueryJobEvent;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class TajoWorkerResourceManager implements WorkerResourceManager {
+ private static final Log LOG = LogFactory.getLog(TajoWorkerResourceManager.class);
+
+ private TajoMaster.MasterContext masterContext;
+
+ private Map<String, WorkerResource> allWorkerResourceMap = new HashMap<String, WorkerResource>();
+ private Set<String> liveWorkerResources = new HashSet<String>();
+ private Set<String> deadWorkerResources = new HashSet<String>();
+
+ private Map<QueryId, WorkerResource> queryMasterMap = new HashMap<QueryId, WorkerResource>();
+
+ private Object workerResourceLock = new Object();
+
+ private final String queryIdSeed;
+
+ private WorkerResourceAllocationThread workerResourceAllocator;
+
+ private final BlockingQueue<WorkerResourceRequest> requestQueue;
+
+ private ReAllocationThread reAllocator;
+
+ private final List<WorkerResourceRequest> reAllocationList;
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ private int queryMasterMemoryMB;
+
+ public TajoWorkerResourceManager(TajoMaster.MasterContext masterContext) {
+ this.masterContext = masterContext;
+ this.queryIdSeed = String.valueOf(System.currentTimeMillis());
+ this.queryMasterMemoryMB = masterContext.getConf().getInt("tajo.querymaster.memoryMB", 512);
+
+ requestQueue = new LinkedBlockingDeque<WorkerResourceRequest>();
+ reAllocationList = new ArrayList<WorkerResourceRequest>();
+
+ workerResourceAllocator = new WorkerResourceAllocationThread();
+ workerResourceAllocator.start();
+
+ reAllocator = new ReAllocationThread();
+ reAllocator.start();
+ }
+
+ @Override
+ public void stop() {
+ if(stopped.get()) {
+ return;
+ }
+ stopped.set(true);
+ if(workerResourceAllocator != null) {
+ workerResourceAllocator.interrupt();
+ }
+
+ if(reAllocator != null) {
+ reAllocator.interrupt();
+ }
+ }
+
+ @Override
+ public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress) {
+ List<WorkerResource> workerResources = chooseWorkers(true, 1, 1, 1);
+ if(workerResources.size() == 0) {
+ //TODO if resource available, assign worker.
+ LOG.warn("No available resource for querymaster:" + queryInProgress.getQueryId());
+ return null;
+ }
+ WorkerResource queryMasterWorker = workerResources.get(0);
+ synchronized(workerResourceLock) {
+ queryMasterMap.put(queryInProgress.getQueryId(), queryMasterWorker);
+ }
+ LOG.info(queryInProgress.getQueryId() + "'s QueryMaster is " + queryMasterWorker);
+ return queryMasterWorker;
+ }
+
+ @Override
+ public void startQueryMaster(QueryInProgress queryInProgress) {
+ WorkerResource queryMasterWorkerResource = null;
+ synchronized(workerResourceLock) {
+ queryMasterWorkerResource = queryMasterMap.get(queryInProgress.getQueryId());
+ }
+
+ if(queryMasterWorkerResource != null) {
+ startQueryMaster(queryInProgress.getQueryId(), queryMasterWorkerResource);
+ } else {
+ //add queue
+ TajoMasterProtocol.WorkerResourceAllocationRequest request =
+ TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+ .setMemoryMBSlots(queryMasterMemoryMB)
+ .setDiskSlots(1)
+ .setExecutionBlockId(QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0).getProto())
+ .setNumWorks(1)
+ .build();
+ try {
+ requestQueue.put(new WorkerResourceRequest(queryInProgress.getQueryId(), true, request, null));
+ } catch (InterruptedException e) {
+ }
+ }
+ }
+
+ private void startQueryMaster(QueryId queryId, WorkerResource workResource) {
+ QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+ if(queryInProgress == null) {
+ LOG.warn("No QueryInProgress while starting QueryMaster:" + queryId);
+ return;
+ }
+ queryInProgress.getQueryInfo().setQueryMasterResource(workResource);
+
+ //fire QueryJobStart event
+ queryInProgress.getEventHandler().handle(
+ new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
+ }
+
+ @Override
+ public String getSeedQueryId() throws IOException {
+ return queryIdSeed;
+ }
+
+ @Override
+ public void allocateWorkerResources(
+ TajoMasterProtocol.WorkerResourceAllocationRequest request,
+ RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack) {
+ try {
+ //TODO checking queue size
+ requestQueue.put(new WorkerResourceRequest(
+ new QueryId(request.getExecutionBlockId().getQueryId()), false, request, callBack));
+ } catch (InterruptedException e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ class WorkerResourceRequest {
+ boolean queryMasterRequest;
+ QueryId queryId;
+ TajoMasterProtocol.WorkerResourceAllocationRequest request;
+ RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack;
+ WorkerResourceRequest(
+ QueryId queryId,
+ boolean queryMasterRequest, TajoMasterProtocol.WorkerResourceAllocationRequest request,
+ RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> callBack) {
+ this.queryId = queryId;
+ this.queryMasterRequest = queryMasterRequest;
+ this.request = request;
+ this.callBack = callBack;
+ }
+ }
+
+ class WorkerResourceAllocationThread extends Thread {
+ @Override
+ public void run() {
+ LOG.info("====>WorkerResourceAllocationThread start");
+ while(!stopped.get()) {
+ try {
+ WorkerResourceRequest resourceRequest = requestQueue.take();
+ List<WorkerResource> workerResources = chooseWorkers(false,
+ resourceRequest.request.getMemoryMBSlots(),
+ resourceRequest.request.getDiskSlots(),
+ resourceRequest.request.getNumWorks());
+
+ LOG.info("====> allocateWorkerResources:" +
+ (new ExecutionBlockId(resourceRequest.request.getExecutionBlockId())) +
+ ", required:" + resourceRequest.request.getNumWorks() + ", allocated:" + workerResources.size() +
+ ", queryMasterRequest=" + resourceRequest.queryMasterRequest +
+ ", liveWorkers=" + liveWorkerResources.size());
+// if(LOG.isDebugEnabled()) {
+// LOG.debug("====> allocateWorkerResources:" +
+// (new ExecutionBlockId(resourceRequest.request.getExecutionBlockId())) +
+// ", required:" + resourceRequest.request.getNumWorks() + ", allocated:" + workerResources.size());
+// } else {
+// LOG.info("====> allocateWorkerResources: required:" + resourceRequest.request.getNumWorks() +
+// ", allocated:" + workerResources.size() + ", queryMasterRequest=" + resourceRequest.queryMasterRequest);
+// }
+
+ if(workerResources.size() > 0) {
+ if(resourceRequest.queryMasterRequest) {
+ startQueryMaster(resourceRequest.queryId, workerResources.get(0));
+ } else {
+ List<String> workerHosts = new ArrayList<String>();
+
+ for(WorkerResource eachWorker: workerResources) {
+ workerHosts.add(eachWorker.getAllocatedHost() + ":" + eachWorker.getPorts()[0]);
+ }
+ resourceRequest.callBack.run(TajoMasterProtocol.WorkerResourceAllocationResponse.newBuilder()
+ .setExecutionBlockId(resourceRequest.request.getExecutionBlockId())
+ .addAllAllocatedWorks(workerHosts)
+ .build()
+ );
+ }
+ } else {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("=========================================");
+ LOG.debug("Available Workers");
+ for(String liveWorker: liveWorkerResources) {
+ LOG.debug(allWorkerResourceMap.get(liveWorker).toString());
+ }
+ LOG.debug("=========================================");
+ }
+ }
+ if(workerResources.size() < resourceRequest.request.getNumWorks()) {
+ reAllocationList.add(new WorkerResourceRequest(
+ resourceRequest.queryId,
+ resourceRequest.queryMasterRequest,
+ TajoMasterProtocol.WorkerResourceAllocationRequest.newBuilder()
+ .setMemoryMBSlots(resourceRequest.request.getMemoryMBSlots())
+ .setDiskSlots(resourceRequest.request.getDiskSlots())
+ .setExecutionBlockId(resourceRequest.request.getExecutionBlockId())
+ .setNumWorks(resourceRequest.request.getNumWorks() - workerResources.size())
+ .build(),
+ resourceRequest.callBack));
+ }
+ } catch(InterruptedException ie) {
+ }
+ }
+ }
+ }
+
+ class ReAllocationThread extends Thread {
+ public void run() {
+ List<WorkerResourceRequest> copiedList = new ArrayList<WorkerResourceRequest>();
+ while(!stopped.get()) {
+ copiedList.clear();
+ synchronized(reAllocationList) {
+ try {
+ reAllocationList.wait(3 * 1000);
+ } catch (InterruptedException e) {
+ if(stopped.get()) {
+ break;
+ }
+ }
+ copiedList.addAll(reAllocationList);
+ }
+
+ for(WorkerResourceRequest eachRequest: copiedList) {
+ try {
+ requestQueue.put(eachRequest);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+ synchronized(reAllocationList) {
+ reAllocationList.clear();
+ }
+ }
+ }
+ }
+
+ private List<WorkerResource> chooseWorkers(boolean queryMaster,
+ int requiredMemoryMBSlots, int requiredDiskSlots,
+ int numWorkers) {
+ List<WorkerResource> selectedWorkers = new ArrayList<WorkerResource>();
+
+ int selectedCount = 0;
+
+ synchronized(workerResourceLock) {
+ for(String eachWorker: liveWorkerResources) {
+ if(selectedCount >= numWorkers) {
+ break;
+ }
+ WorkerResource workerResource = allWorkerResourceMap.get(eachWorker);
+ if(workerResource.getAvailableMemoryMBSlots() >= requiredMemoryMBSlots &&
+ workerResource.getAvailableDiskSlots() >= requiredDiskSlots) {
+ if(queryMaster && workerResource.isQueryMasterAllocated()) {
+ continue;
+ }
+ workerResource.addUsedMemoryMBSlots(requiredMemoryMBSlots);
+ workerResource.addUsedDiskSlots(requiredDiskSlots);
+ workerResource.setQueryMasterAllocated(queryMaster);
+ selectedWorkers.add(workerResource);
+ selectedCount++;
+ }
+ }
+ }
+
+ return selectedWorkers;
+ }
+
+ public Collection<WorkerResource> getClusterWorkResources() {
+ return Collections.unmodifiableCollection(allWorkerResourceMap.values());
+ }
+
+ @Override
+ public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource) {
+ synchronized(workerResourceLock) {
+ WorkerResource managedWorkerResource = allWorkerResourceMap.get(workerResource.getId());
+ if(managedWorkerResource != null) {
+ managedWorkerResource.releaseResource(workerResource);
+ }
+ }
+
+ synchronized(reAllocationList) {
+ reAllocationList.notifyAll();
+ }
+ }
+
+ @Override
+ public boolean isQueryMasterStopped(QueryId queryId) {
+ synchronized(workerResourceLock) {
+ return !queryMasterMap.containsKey(queryId);
+ }
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ }
+
+ @Override
+ public void stopQueryMaster(QueryId queryId) {
+ WorkerResource workerResource = null;
+ synchronized(workerResourceLock) {
+ workerResource = queryMasterMap.remove(queryId);
+ }
+ LOG.info("release QueryMaster resource:" + queryId + "," + workerResource.isQueryMasterAllocated());
+ if(workerResource != null) {
+ releaseWorkerResource(queryId, workerResource);
+ }
+ }
+
+ public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request) {
+ synchronized(workerResourceLock) {
+ String hostAndPort = request.getTajoWorkerHost() + ":" + request.getTajoWorkerPort();
+ if(allWorkerResourceMap.containsKey(hostAndPort)) {
+ if(deadWorkerResources.contains(hostAndPort)) {
+ deadWorkerResources.remove(hostAndPort);
+ liveWorkerResources.add(hostAndPort);
+ }
+ WorkerResource workerResource = allWorkerResourceMap.get(hostAndPort);
+ workerResource.setLastHeartbeat(System.currentTimeMillis());
+ workerResource.setWorkerStatus(WorkerStatus.LIVE);
+ } else {
+ WorkerResource workerResource = new WorkerResource();
+ workerResource.setAllocatedHost(request.getTajoWorkerHost());
+
+ int[] ports = new int[] { request.getTajoWorkerPort(), request.getTajoWorkerClientPort() };
+
+ workerResource.setPorts(ports);
+
+ workerResource.setLastHeartbeat(System.currentTimeMillis());
+ workerResource.setWorkerStatus(WorkerStatus.LIVE);
+ if(request.getServerStatus() != null) {
+ workerResource.setMemoryMBSlots(request.getServerStatus().getSystem().getTotalMemoryMB());
+ workerResource.setCpuCoreSlots(request.getServerStatus().getSystem().getAvailableProcessors());
+ workerResource.setDiskSlots(request.getServerStatus().getDiskSlots());
+ } else {
+ workerResource.setMemoryMBSlots(4096);
+ workerResource.setDiskSlots(4);
+ workerResource.setCpuCoreSlots(4);
+ }
+
+ allWorkerResourceMap.put(workerResource.getId(), workerResource);
+ liveWorkerResources.add(hostAndPort);
+
+ LOG.info("====> TajoWorker:" + workerResource + " added in live TajoWorker list");
+
+ workerResourceLock.notifyAll();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
new file mode 100644
index 0000000..00bd509
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResource.java
@@ -0,0 +1,194 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class WorkerResource {
+ private static final Log LOG = LogFactory.getLog(WorkerResource.class);
+
+ private String allocatedHost;
+ private int[] ports;
+
+ private int diskSlots;
+ private int cpuCoreSlots;
+ private int memoryMBSlots;
+
+ private int usedDiskSlots;
+ private int usedMemoryMBSlots;
+ private int usedCpuCoreSlots;
+
+ private boolean queryMasterAllocated;
+
+ private WorkerStatus workerStatus;
+
+ private long lastHeartbeat;
+
+ public String getId() {
+ if(ports.length > 0) {
+ return allocatedHost + ":" + ports[0];
+ } else {
+ return allocatedHost;
+ }
+ }
+
+ public String getAllocatedHost() {
+ return allocatedHost;
+ }
+
+ public void setAllocatedHost(String allocatedHost) {
+ this.allocatedHost = allocatedHost;
+ }
+
+ public int[] getPorts() {
+ return ports;
+ }
+
+ public void setPorts(int[] ports) {
+ this.ports = ports;
+ }
+
+ public void addUsedDiskSlots(int diskSlots) {
+ usedDiskSlots += diskSlots;
+ }
+
+ public void addUsedMemoryMBSlots(int memoryMBSlots) {
+ usedMemoryMBSlots += memoryMBSlots;
+ }
+
+ public void addUsedCpuCoreSlots(int cpuCoreSlots) {
+ usedCpuCoreSlots += cpuCoreSlots;
+ }
+
+ public int getDiskSlots() {
+ return diskSlots;
+ }
+
+ public void setDiskSlots(int diskSlots) {
+ this.diskSlots = diskSlots;
+ }
+
+ public int getCpuCoreSlots() {
+ return cpuCoreSlots;
+ }
+
+ public void setCpuCoreSlots(int cpuCoreSlots) {
+ this.cpuCoreSlots = cpuCoreSlots;
+ }
+
+ public int getMemoryMBSlots() {
+ return memoryMBSlots;
+ }
+
+ public void setMemoryMBSlots(int memoryMBSlots) {
+ this.memoryMBSlots = memoryMBSlots;
+ }
+
+ public int getAvailableDiskSlots() {
+ return diskSlots - usedDiskSlots;
+ }
+
+ public int getAvailableMemoryMBSlots() {
+ return memoryMBSlots - usedMemoryMBSlots;
+ }
+
+ @Override
+ public String toString() {
+ return "host:" + allocatedHost + ", port=" + portsToStr() + ", slots=" + memoryMBSlots + ":" + cpuCoreSlots + ":" + diskSlots +
+ ", used=" + usedMemoryMBSlots + ":" + usedCpuCoreSlots + ":" + usedDiskSlots;
+ }
+
+ private String portsToStr() {
+ if(ports == null) {
+ return "null";
+ }
+ String result = "";
+ String prefix = "";
+ for(int i = 0; i < ports.length; i++) {
+ result += prefix + ports[i];
+ prefix = ",";
+ }
+
+ return result;
+ }
+
+ public void setLastHeartbeat(long heartbeatTime) {
+ this.lastHeartbeat = heartbeatTime;
+ }
+
+ public int getUsedMemoryMBSlots() {
+ return usedMemoryMBSlots;
+ }
+
+ public void setUsedMemoryMBSlots(int usedMemoryMBSlots) {
+ this.usedMemoryMBSlots = usedMemoryMBSlots;
+ }
+
+ public int getUsedCpuCoreSlots() {
+ return usedCpuCoreSlots;
+ }
+
+ public void setUsedCpuCoreSlots(int usedCpuCoreSlots) {
+ this.usedCpuCoreSlots = usedCpuCoreSlots;
+ }
+
+ public int getUsedDiskSlots() {
+ return usedDiskSlots;
+ }
+
+ public void setUsedDiskSlots(int usedDiskSlots) {
+ this.usedDiskSlots = usedDiskSlots;
+ }
+
+ public WorkerStatus getWorkerStatus() {
+ return workerStatus;
+ }
+
+ public void setWorkerStatus(WorkerStatus workerStatus) {
+ this.workerStatus = workerStatus;
+ }
+
+ public long getLastHeartbeat() {
+ return lastHeartbeat;
+ }
+
+ public boolean isQueryMasterAllocated() {
+ return queryMasterAllocated;
+ }
+
+ public void setQueryMasterAllocated(boolean queryMasterAllocated) {
+ this.queryMasterAllocated = queryMasterAllocated;
+ }
+
+ public void releaseResource(WorkerResource workerResource) {
+ if(workerResource.isQueryMasterAllocated()) {
+ queryMasterAllocated = false;
+ }
+
+ usedMemoryMBSlots -= workerResource.memoryMBSlots;
+ usedDiskSlots -= workerResource.diskSlots;
+
+ if(usedMemoryMBSlots < 0 || usedDiskSlots < 0 || usedCpuCoreSlots < 0) {
+// LOG.warn("Used resources can't be a minus.");
+ LOG.trace("Used resources can't be a minus.");
+ LOG.warn(this + " ==> " + workerResource);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
new file mode 100644
index 0000000..a066321
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.querymaster.QueryInProgress;
+
+import java.io.IOException;
+
+public interface WorkerResourceManager {
+
+ /**
+ * select Worker for QueryMaster
+ * @param queryInProgress
+ * @return
+ */
+ public WorkerResource allocateQueryMaster(QueryInProgress queryInProgress);
+
+ public void allocateWorkerResources(TajoMasterProtocol.WorkerResourceAllocationRequest request,
+ RpcCallback<TajoMasterProtocol.WorkerResourceAllocationResponse> rpcCallBack);
+
+ /**
+ * start Worker for query master(YARN) or assign query master role(Standby Mode)
+ * @param queryInProgress
+ */
+ public void startQueryMaster(QueryInProgress queryInProgress);
+
+ public String getSeedQueryId() throws IOException;
+
+ public boolean isQueryMasterStopped(QueryId queryId);
+
+ public void init(Configuration conf);
+
+ public void stopQueryMaster(QueryId queryId);
+
+ public void workerHeartbeat(TajoMasterProtocol.TajoHeartbeat request);
+
+ public void releaseWorkerResource(QueryId queryId, WorkerResource workerResource);
+
+ public void stop();
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java
new file mode 100644
index 0000000..21ad7d7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/WorkerStatus.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+public enum WorkerStatus {
+ LIVE,
+ DEAD,
+ DECOMMISSION
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/d48f2667/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
new file mode 100644
index 0000000..d7adad8
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/rm/YarnRMContainerAllocator.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master.rm;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.AMResponse;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.client.AMRMClientImpl;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.master.event.ContainerAllocationEvent;
+import org.apache.tajo.master.event.ContainerAllocatorEventType;
+import org.apache.tajo.master.event.SubQueryContainerAllocationEvent;
+import org.apache.tajo.master.querymaster.Query;
+import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.master.querymaster.SubQuery;
+import org.apache.tajo.master.querymaster.SubQueryState;
+import org.apache.tajo.util.ApplicationIdUtils;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class YarnRMContainerAllocator extends AMRMClientImpl
+ implements EventHandler<ContainerAllocationEvent> {
+
+ /** Class Logger */
+ private static final Log LOG = LogFactory.getLog(YarnRMContainerAllocator.
+ class.getName());
+
+ private QueryMasterTask.QueryContext context;
+ private final EventHandler eventHandler;
+
+ public YarnRMContainerAllocator(QueryMasterTask.QueryContext context) {
+ super(ApplicationIdUtils.createApplicationAttemptId(context.getQueryId()));
+ this.context = context;
+ this.eventHandler = context.getDispatcher().getEventHandler();
+ }
+
+ public void init(Configuration conf) {
+ super.init(conf);
+ }
+
+ private static final int WAIT_INTERVAL_AVAILABLE_NODES = 500; // 0.5 second
+ public void start() {
+ super.start();
+
+ RegisterApplicationMasterResponse response;
+ try {
+ response = registerApplicationMaster("localhost", 10080, "http://localhost:1234");
+ context.getResourceAllocator().setMaxContainerCapability(response.getMaximumResourceCapability().getMemory());
+ context.getResourceAllocator().setMinContainerCapability(response.getMinimumResourceCapability().getMemory());
+
+ // If the number of cluster nodes is ZERO, it waits for available nodes.
+ AllocateResponse allocateResponse = allocate(0.0f);
+ while(allocateResponse.getNumClusterNodes() < 1) {
+ try {
+ Thread.sleep(WAIT_INTERVAL_AVAILABLE_NODES);
+ LOG.info("Waiting for Available Cluster Nodes");
+ allocateResponse = allocate(0);
+ } catch (InterruptedException e) {
+ LOG.error(e);
+ }
+ }
+ context.getResourceAllocator().setNumClusterNodes(allocateResponse.getNumClusterNodes());
+ } catch (YarnRemoteException e) {
+ LOG.error(e);
+ }
+
+ startAllocatorThread();
+ }
+
+ protected Thread allocatorThread;
+ private final AtomicBoolean stopped = new AtomicBoolean(false);
+ private int rmPollInterval = 1000;//millis
+
+ protected void startAllocatorThread() {
+ allocatorThread = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+ try {
+ try {
+ heartbeat();
+ } catch (YarnException e) {
+ LOG.error("Error communicating with RM: " + e.getMessage() , e);
+ return;
+ } catch (Exception e) {
+ LOG.error("ERROR IN CONTACTING RM. ", e);
+ // TODO: for other exceptions
+ if(stopped.get()) {
+ break;
+ }
+ }
+ Thread.sleep(rmPollInterval);
+ } catch (InterruptedException e) {
+ if (!stopped.get()) {
+ LOG.warn("Allocated thread interrupted. Returning.");
+ }
+ break;
+ }
+ }
+ LOG.info("Allocated thread stopped");
+ }
+ });
+ allocatorThread.setName("YarnRMContainerAllocator");
+ allocatorThread.start();
+ }
+
+ public void stop() {
+ if(stopped.get()) {
+ return;
+ }
+ LOG.info("un-registering ApplicationMaster(QueryMaster):" + appAttemptId);
+ stopped.set(true);
+
+ try {
+ FinalApplicationStatus status = FinalApplicationStatus.UNDEFINED;
+ Query query = context.getQuery();
+ if (query != null) {
+ TajoProtos.QueryState state = query.getState();
+ if (state == TajoProtos.QueryState.QUERY_SUCCEEDED) {
+ status = FinalApplicationStatus.SUCCEEDED;
+ } else if (state == TajoProtos.QueryState.QUERY_FAILED || state == TajoProtos.QueryState.QUERY_ERROR) {
+ status = FinalApplicationStatus.FAILED;
+ } else if (state == TajoProtos.QueryState.QUERY_ERROR) {
+ status = FinalApplicationStatus.FAILED;
+ }
+ }
+ unregisterApplicationMaster(status, "tajo query finished", null);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+
+ allocatorThread.interrupt();
+ LOG.info("un-registered ApplicationMAster(QueryMaster) stopped:" + appAttemptId);
+
+ super.stop();
+ }
+
+ private final Map<Priority, ExecutionBlockId> subQueryMap =
+ new HashMap<Priority, ExecutionBlockId>();
+
+ public void heartbeat() throws Exception {
+ AllocateResponse allocateResponse = allocate(context.getProgress());
+ AMResponse response = allocateResponse.getAMResponse();
+ if(response == null) {
+ LOG.warn("AM Response is null");
+ return;
+ }
+ List<Container> allocatedContainers = response.getAllocatedContainers();
+
+ LOG.info("Available Cluster Nodes: " + allocateResponse.getNumClusterNodes());
+ LOG.info("Available Resource: " + response.getAvailableResources());
+ LOG.info("Num of Allocated Containers: " + response.getAllocatedContainers().size());
+ if (response.getAllocatedContainers().size() > 0) {
+ LOG.info("================================================================");
+ for (Container container : response.getAllocatedContainers()) {
+ LOG.info("> Container Id: " + container.getId());
+ LOG.info("> Node Id: " + container.getNodeId());
+ LOG.info("> Resource (Mem): " + container.getResource().getMemory());
+ LOG.info("> State : " + container.getState());
+ LOG.info("> Priority: " + container.getPriority());
+ }
+ LOG.info("================================================================");
+ }
+
+ Map<ExecutionBlockId, List<Container>> allocated = new HashMap<ExecutionBlockId, List<Container>>();
+ if (allocatedContainers.size() > 0) {
+ for (Container container : allocatedContainers) {
+ ExecutionBlockId executionBlockId = subQueryMap.get(container.getPriority());
+ SubQueryState state = context.getSubQuery(executionBlockId).getState();
+ if (!(SubQuery.isRunningState(state) && subQueryMap.containsKey(container.getPriority()))) {
+ releaseAssignedContainer(container.getId());
+ synchronized (subQueryMap) {
+ subQueryMap.remove(container.getPriority());
+ }
+ } else {
+ if (allocated.containsKey(executionBlockId)) {
+ allocated.get(executionBlockId).add(container);
+ } else {
+ allocated.put(executionBlockId, Lists.newArrayList(container));
+ }
+ }
+ }
+
+ for (Entry<ExecutionBlockId, List<Container>> entry : allocated.entrySet()) {
+ eventHandler.handle(new SubQueryContainerAllocationEvent(entry.getKey(), entry.getValue()));
+ }
+ }
+ }
+
+ @Override
+ public void handle(ContainerAllocationEvent event) {
+
+ if (event.getType() == ContainerAllocatorEventType.CONTAINER_REQ) {
+ LOG.info(event);
+ subQueryMap.put(event.getPriority(), event.getExecutionBlockId());
+ addContainerRequest(new ContainerRequest(event.getCapability(), null, null,
+ event.getPriority(), event.getRequiredNum()));
+
+ } else if (event.getType() == ContainerAllocatorEventType.CONTAINER_DEALLOCATE) {
+ LOG.info(event);
+ } else {
+ LOG.info(event);
+ }
+ }
+}